HDFS-12415. Ozone: TestXceiverClientManager and TestAllocateContainer occasionally fails. Contributed by Mukul Kumar Singh.
This commit is contained in:
parent
895a4cc60a
commit
fb545e4291
|
@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.scm.node;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import org.apache.commons.collections.map.HashedMap;
|
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||||
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
|
||||||
import org.apache.hadoop.metrics2.util.MBeans;
|
import org.apache.hadoop.metrics2.util.MBeans;
|
||||||
|
@ -105,13 +104,13 @@ public class SCMNodeManager
|
||||||
/**
|
/**
|
||||||
* Key = NodeID, value = timestamp.
|
* Key = NodeID, value = timestamp.
|
||||||
*/
|
*/
|
||||||
private final Map<String, Long> healthyNodes;
|
private final ConcurrentHashMap<String, Long> healthyNodes;
|
||||||
private final Map<String, Long> staleNodes;
|
private final ConcurrentHashMap<String, Long> staleNodes;
|
||||||
private final Map<String, Long> deadNodes;
|
private final ConcurrentHashMap<String, Long> deadNodes;
|
||||||
private final Queue<HeartbeatQueueItem> heartbeatQueue;
|
private final Queue<HeartbeatQueueItem> heartbeatQueue;
|
||||||
private final Map<String, DatanodeID> nodes;
|
private final ConcurrentHashMap<String, DatanodeID> nodes;
|
||||||
// Individual live node stats
|
// Individual live node stats
|
||||||
private final Map<String, SCMNodeStat> nodeStats;
|
private final ConcurrentHashMap<String, SCMNodeStat> nodeStats;
|
||||||
// Aggregated node stats
|
// Aggregated node stats
|
||||||
private SCMNodeStat scmStat;
|
private SCMNodeStat scmStat;
|
||||||
// TODO: expose nodeStats and scmStat as metrics
|
// TODO: expose nodeStats and scmStat as metrics
|
||||||
|
@ -158,8 +157,8 @@ public class SCMNodeManager
|
||||||
healthyNodes = new ConcurrentHashMap<>();
|
healthyNodes = new ConcurrentHashMap<>();
|
||||||
deadNodes = new ConcurrentHashMap<>();
|
deadNodes = new ConcurrentHashMap<>();
|
||||||
staleNodes = new ConcurrentHashMap<>();
|
staleNodes = new ConcurrentHashMap<>();
|
||||||
nodes = new HashMap<>();
|
nodes = new ConcurrentHashMap<>();
|
||||||
nodeStats = new HashedMap();
|
nodeStats = new ConcurrentHashMap();
|
||||||
scmStat = new SCMNodeStat();
|
scmStat = new SCMNodeStat();
|
||||||
|
|
||||||
healthyNodeCount = new AtomicInteger(0);
|
healthyNodeCount = new AtomicInteger(0);
|
||||||
|
|
Loading…
Reference in New Issue