|
@@ -23,39 +23,31 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
|
|
|
|
|
|
+import java.util.Collections;
|
|
|
|
+import java.util.HashSet;
|
|
|
|
+import java.util.Map;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
import java.util.UUID;
|
|
import java.util.UUID;
|
|
-import java.util.Map;
|
|
|
|
-import java.util.HashSet;
|
|
|
|
-import java.util.Collections;
|
|
|
|
-
|
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
-import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
|
|
|
- .DUPLICATE_DATANODE;
|
|
|
|
-
|
|
|
|
|
|
+import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.DUPLICATE_DATANODE;
|
|
|
|
|
|
/**
|
|
/**
|
|
- * This data structure maintains the list of pipelines which the given datanode
|
|
|
|
- * is a part of.
|
|
|
|
- * This information will be added whenever a new pipeline allocation happens.
|
|
|
|
|
|
+ * This data structure maintains the list of pipelines which the given datanode is a part of. This
|
|
|
|
+ * information will be added whenever a new pipeline allocation happens.
|
|
*
|
|
*
|
|
- * TODO: this information needs to be regenerated from pipeline reports on
|
|
|
|
- * SCM restart
|
|
|
|
|
|
+ * <p>TODO: this information needs to be regenerated from pipeline reports on SCM restart
|
|
*/
|
|
*/
|
|
public class Node2PipelineMap {
|
|
public class Node2PipelineMap {
|
|
private final Map<UUID, Set<Pipeline>> dn2PipelineMap;
|
|
private final Map<UUID, Set<Pipeline>> dn2PipelineMap;
|
|
|
|
|
|
- /**
|
|
|
|
- * Constructs a Node2PipelineMap Object.
|
|
|
|
- */
|
|
|
|
|
|
+ /** Constructs a Node2PipelineMap Object. */
|
|
public Node2PipelineMap() {
|
|
public Node2PipelineMap() {
|
|
dn2PipelineMap = new ConcurrentHashMap<>();
|
|
dn2PipelineMap = new ConcurrentHashMap<>();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Returns true if this a datanode that is already tracked by
|
|
|
|
- * Node2PipelineMap.
|
|
|
|
|
|
+ * Returns true if this a datanode that is already tracked by Node2PipelineMap.
|
|
*
|
|
*
|
|
* @param datanodeID - UUID of the Datanode.
|
|
* @param datanodeID - UUID of the Datanode.
|
|
* @return True if this is tracked, false if this map does not know about it.
|
|
* @return True if this is tracked, false if this map does not know about it.
|
|
@@ -71,18 +63,17 @@ public class Node2PipelineMap {
|
|
* @param datanodeID -- Datanode UUID
|
|
* @param datanodeID -- Datanode UUID
|
|
* @param pipelines - set of pipelines.
|
|
* @param pipelines - set of pipelines.
|
|
*/
|
|
*/
|
|
- private void insertNewDatanode(UUID datanodeID, Set<Pipeline> pipelines)
|
|
|
|
- throws SCMException {
|
|
|
|
|
|
+ private void insertNewDatanode(UUID datanodeID, Set<Pipeline> pipelines) throws SCMException {
|
|
Preconditions.checkNotNull(pipelines);
|
|
Preconditions.checkNotNull(pipelines);
|
|
Preconditions.checkNotNull(datanodeID);
|
|
Preconditions.checkNotNull(datanodeID);
|
|
- if(dn2PipelineMap.putIfAbsent(datanodeID, pipelines) != null) {
|
|
|
|
- throw new SCMException("Node already exists in the map",
|
|
|
|
- DUPLICATE_DATANODE);
|
|
|
|
|
|
+ if (dn2PipelineMap.putIfAbsent(datanodeID, pipelines) != null) {
|
|
|
|
+ throw new SCMException("Node already exists in the map", DUPLICATE_DATANODE);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Removes datanode Entry from the map.
|
|
* Removes datanode Entry from the map.
|
|
|
|
+ *
|
|
* @param datanodeID - Datanode ID.
|
|
* @param datanodeID - Datanode ID.
|
|
*/
|
|
*/
|
|
public synchronized void removeDatanode(UUID datanodeID) {
|
|
public synchronized void removeDatanode(UUID datanodeID) {
|
|
@@ -98,20 +89,19 @@ public class Node2PipelineMap {
|
|
*/
|
|
*/
|
|
public Set<Pipeline> getPipelines(UUID datanode) {
|
|
public Set<Pipeline> getPipelines(UUID datanode) {
|
|
Preconditions.checkNotNull(datanode);
|
|
Preconditions.checkNotNull(datanode);
|
|
- return dn2PipelineMap.computeIfPresent(datanode, (k, v) ->
|
|
|
|
- Collections.unmodifiableSet(v));
|
|
|
|
|
|
+ return dn2PipelineMap.computeIfPresent(datanode, (k, v) -> Collections.unmodifiableSet(v));
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Adds a pipeline entry to a given dataNode in the map.
|
|
* Adds a pipeline entry to a given dataNode in the map.
|
|
|
|
+ *
|
|
* @param pipeline Pipeline to be added
|
|
* @param pipeline Pipeline to be added
|
|
*/
|
|
*/
|
|
public synchronized void addPipeline(Pipeline pipeline) {
|
|
public synchronized void addPipeline(Pipeline pipeline) {
|
|
for (DatanodeDetails details : pipeline.getDatanodes().values()) {
|
|
for (DatanodeDetails details : pipeline.getDatanodes().values()) {
|
|
UUID dnId = details.getUuid();
|
|
UUID dnId = details.getUuid();
|
|
dn2PipelineMap
|
|
dn2PipelineMap
|
|
- .computeIfAbsent(dnId,
|
|
|
|
- k -> Collections.synchronizedSet(new HashSet<>()))
|
|
|
|
|
|
+ .computeIfAbsent(dnId, k -> Collections.synchronizedSet(new HashSet<>()))
|
|
.add(pipeline);
|
|
.add(pipeline);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -119,8 +109,12 @@ public class Node2PipelineMap {
|
|
public synchronized void removePipeline(Pipeline pipeline) {
|
|
public synchronized void removePipeline(Pipeline pipeline) {
|
|
for (DatanodeDetails details : pipeline.getDatanodes().values()) {
|
|
for (DatanodeDetails details : pipeline.getDatanodes().values()) {
|
|
UUID dnId = details.getUuid();
|
|
UUID dnId = details.getUuid();
|
|
- dn2PipelineMap.computeIfPresent(dnId,
|
|
|
|
- (k, v) -> {v.remove(pipeline); return v;});
|
|
|
|
|
|
+ dn2PipelineMap.computeIfPresent(
|
|
|
|
+ dnId,
|
|
|
|
+ (k, v) -> {
|
|
|
|
+ v.remove(pipeline);
|
|
|
|
+ return v;
|
|
|
|
+ });
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|