Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -729,9 +729,8 @@ public void run() {
*/
public synchronized void forceNodesToHealthyReadOnly() {
try {
List<UUID> nodes = nodeStateMap.getNodes(null, HEALTHY);
for (UUID id : nodes) {
DatanodeInfo node = nodeStateMap.getNodeInfo(id);
List<DatanodeInfo> nodes = nodeStateMap.filterNodes(null, HEALTHY);
for (DatanodeInfo node : nodes) {
nodeStateMap.updateNodeHealthState(node.getUuid(),
HEALTHY_READONLY);
if (state2EventMap.containsKey(HEALTHY_READONLY)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,14 @@

import java.util.ArrayList;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.Collectors;
import java.util.function.Predicate;

import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
Expand Down Expand Up @@ -182,21 +183,6 @@ public DatanodeInfo getNodeInfo(UUID uuid) throws NodeNotFoundException {
}
}

/**
* Returns the list of node ids which are in the specified state.
*
* @param status NodeStatus
*
* @return list of node ids
*/
public List<UUID> getNodes(NodeStatus status) {
ArrayList<UUID> nodes = new ArrayList<>();
for (DatanodeInfo dn : filterNodes(status)) {
nodes.add(dn.getUuid());
}
return nodes;
}

/**
* Returns the list of node ids which match the desired operational state
* and health. Passing a null for either value is equivalent to a wild card.
Expand Down Expand Up @@ -252,7 +238,7 @@ public List<DatanodeInfo> getAllDatanodeInfos() {
* @return List of DatanodeInfo for the matching nodes
*/
public List<DatanodeInfo> getDatanodeInfos(NodeStatus status) {
return filterNodes(status);
return filterNodes(matching(status));
}

/**
Expand All @@ -277,7 +263,7 @@ public List<DatanodeInfo> getDatanodeInfos(
* @return Number of nodes in the specified state
*/
public int getNodeCount(NodeStatus state) {
return getNodes(state).size();
return getDatanodeInfos(state).size();
}

/**
Expand Down Expand Up @@ -427,41 +413,47 @@ private void checkIfNodeExist(UUID uuid) throws NodeNotFoundException {
* @param health
* @return List of DatanodeInfo objects matching the passed state
*/
private List<DatanodeInfo> filterNodes(
public List<DatanodeInfo> filterNodes(
NodeOperationalState opState, NodeState health) {
if (opState != null && health != null) {
return filterNodes(new NodeStatus(opState, health));
return filterNodes(matching(new NodeStatus(opState, health)));
}
if (opState == null && health == null) {
return getAllDatanodeInfos();
if (opState != null) {
return filterNodes(matching(opState));
}
try {
lock.readLock().lock();
return nodeMap.values().stream()
.filter(n -> opState == null
|| n.getNodeStatus().getOperationalState() == opState)
.filter(n -> health == null
|| n.getNodeStatus().getHealth() == health)
.collect(Collectors.toList());
} finally {
lock.readLock().unlock();
if (health != null) {
return filterNodes(matching(health));
}
return getAllDatanodeInfos();
}

/**
* Create a list of datanodeInfo for all nodes matching the passsed status.
*
* @param status
* @return List of DatanodeInfo objects matching the passed state
* @return a list of all nodes matching the {@code filter}
*/
private List<DatanodeInfo> filterNodes(NodeStatus status) {
private List<DatanodeInfo> filterNodes(Predicate<DatanodeInfo> filter) {
List<DatanodeInfo> result = new LinkedList<>();
lock.readLock().lock();
try {
lock.readLock().lock();
return nodeMap.values().stream()
.filter(n -> n.getNodeStatus().equals(status))
.collect(Collectors.toList());
} finally {
for (DatanodeInfo dn : nodeMap.values()) {
if (filter.test(dn)) {
result.add(dn);
}
}
} finally {
lock.readLock().unlock();
}
return result;
}

private static Predicate<DatanodeInfo> matching(NodeStatus status) {
return dn -> status.equals(dn.getNodeStatus());
}

private static Predicate<DatanodeInfo> matching(NodeOperationalState state) {
return dn -> state.equals(dn.getNodeStatus().getOperationalState());
}

private static Predicate<DatanodeInfo> matching(NodeState health) {
return dn -> health.equals(dn.getNodeStatus().getHealth());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
import org.apache.hadoop.hdds.scm.node.NodeStatus;

import static org.junit.jupiter.api.Assertions.assertEquals;
Expand Down Expand Up @@ -105,7 +106,7 @@ public void testGetNodeMethodsReturnCorrectCountsAndStates()
}
}
NodeStatus requestedState = NodeStatus.inServiceStale();
List<UUID> nodes = map.getNodes(requestedState);
List<DatanodeInfo> nodes = map.getDatanodeInfos(requestedState);
assertEquals(1, nodes.size());
assertEquals(1, map.getNodeCount(requestedState));
assertEquals(nodeCount, map.getTotalNodeCount());
Expand Down