|
@@ -16,9 +16,12 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdds.scm.pipelines;
|
|
|
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
|
+import org.apache.hadoop.hdds.protocol.proto
|
|
|
+ .StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
|
|
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
|
@@ -30,6 +33,7 @@ import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
|
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
|
|
+import org.apache.hadoop.hdds.scm.node.states.Node2PipelineMap;
|
|
|
import org.apache.hadoop.hdds.scm.pipelines.ratis.RatisManagerImpl;
|
|
|
import org.apache.hadoop.hdds.scm.pipelines.standalone.StandaloneManagerImpl;
|
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
@@ -75,11 +79,9 @@ public class PipelineSelector {
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(PipelineSelector.class);
|
|
|
private final ContainerPlacementPolicy placementPolicy;
|
|
|
- private final NodeManager nodeManager;
|
|
|
+ private final Map<ReplicationType, PipelineManager> pipelineManagerMap;
|
|
|
private final Configuration conf;
|
|
|
private final EventPublisher eventPublisher;
|
|
|
- private final RatisManagerImpl ratisManager;
|
|
|
- private final StandaloneManagerImpl standaloneManager;
|
|
|
private final long containerSize;
|
|
|
private final MetadataStore pipelineStore;
|
|
|
private final PipelineStateManager stateManager;
|
|
@@ -96,7 +98,6 @@ public class PipelineSelector {
|
|
|
*/
|
|
|
public PipelineSelector(NodeManager nodeManager, Configuration conf,
|
|
|
EventPublisher eventPublisher, int cacheSizeMB) throws IOException {
|
|
|
- this.nodeManager = nodeManager;
|
|
|
this.conf = conf;
|
|
|
this.eventPublisher = eventPublisher;
|
|
|
this.placementPolicy = createContainerPlacementPolicy(nodeManager, conf);
|
|
@@ -106,12 +107,14 @@ public class PipelineSelector {
|
|
|
StorageUnit.BYTES);
|
|
|
node2PipelineMap = new Node2PipelineMap();
|
|
|
pipelineMap = new ConcurrentHashMap<>();
|
|
|
- this.standaloneManager =
|
|
|
- new StandaloneManagerImpl(this.nodeManager, placementPolicy,
|
|
|
- containerSize);
|
|
|
- this.ratisManager =
|
|
|
- new RatisManagerImpl(this.nodeManager, placementPolicy, containerSize,
|
|
|
- conf);
|
|
|
+ pipelineManagerMap = new HashMap<>();
|
|
|
+
|
|
|
+ pipelineManagerMap.put(ReplicationType.STAND_ALONE,
|
|
|
+ new StandaloneManagerImpl(nodeManager, placementPolicy,
|
|
|
+ containerSize));
|
|
|
+ pipelineManagerMap.put(ReplicationType.RATIS,
|
|
|
+ new RatisManagerImpl(nodeManager, placementPolicy,
|
|
|
+ containerSize, conf));
|
|
|
long pipelineCreationLeaseTimeout = conf.getTimeDuration(
|
|
|
ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT,
|
|
|
ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT_DEFAULT,
|
|
@@ -154,6 +157,7 @@ public class PipelineSelector {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
public Set<ContainerID> getOpenContainerIDsByPipeline(PipelineID pipelineID) {
|
|
|
return pipeline2ContainerMap.get(pipelineID);
|
|
|
}
|
|
@@ -226,30 +230,6 @@ public class PipelineSelector {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Return the pipeline manager from the replication type.
|
|
|
- *
|
|
|
- * @param replicationType - Replication Type Enum.
|
|
|
- * @return pipeline Manager.
|
|
|
- * @throws IllegalArgumentException If an pipeline type gets added
|
|
|
- * and this function is not modified we will throw.
|
|
|
- */
|
|
|
- private PipelineManager getPipelineManager(ReplicationType replicationType)
|
|
|
- throws IllegalArgumentException {
|
|
|
- switch (replicationType) {
|
|
|
- case RATIS:
|
|
|
- return this.ratisManager;
|
|
|
- case STAND_ALONE:
|
|
|
- return this.standaloneManager;
|
|
|
- case CHAINED:
|
|
|
- throw new IllegalArgumentException("Not implemented yet");
|
|
|
- default:
|
|
|
- throw new IllegalArgumentException("Unexpected enum found. Does not" +
|
|
|
- " know how to handle " + replicationType.toString());
|
|
|
- }
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* This function is called by the Container Manager while allocating a new
|
|
|
* container. The client specifies what kind of replication pipeline is needed
|
|
@@ -260,7 +240,7 @@ public class PipelineSelector {
|
|
|
public Pipeline getReplicationPipeline(ReplicationType replicationType,
|
|
|
HddsProtos.ReplicationFactor replicationFactor)
|
|
|
throws IOException {
|
|
|
- PipelineManager manager = getPipelineManager(replicationType);
|
|
|
+ PipelineManager manager = pipelineManagerMap.get(replicationType);
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
LOG.debug("Getting replication pipeline forReplicationType {} :" +
|
|
|
" ReplicationFactor {}", replicationType.toString(),
|
|
@@ -316,7 +296,7 @@ public class PipelineSelector {
|
|
|
* Finalize a given pipeline.
|
|
|
*/
|
|
|
public void finalizePipeline(Pipeline pipeline) throws IOException {
|
|
|
- PipelineManager manager = getPipelineManager(pipeline.getType());
|
|
|
+ PipelineManager manager = pipelineManagerMap.get(pipeline.getType());
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
if (pipeline.getLifeCycleState() == LifeCycleState.CLOSING ||
|
|
|
pipeline.getLifeCycleState() == LifeCycleState.CLOSED) {
|
|
@@ -327,17 +307,17 @@ public class PipelineSelector {
|
|
|
}
|
|
|
|
|
|
// Remove the pipeline from active allocation
|
|
|
- manager.finalizePipeline(pipeline);
|
|
|
-
|
|
|
- LOG.info("Finalizing pipeline. pipelineID: {}", pipeline.getId());
|
|
|
- updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.FINALIZE);
|
|
|
- closePipelineIfNoOpenContainers(pipeline);
|
|
|
+ if (manager.finalizePipeline(pipeline)) {
|
|
|
+ LOG.info("Finalizing pipeline. pipelineID: {}", pipeline.getId());
|
|
|
+ updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.FINALIZE);
|
|
|
+ closePipelineIfNoOpenContainers(pipeline);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Close a given pipeline.
|
|
|
*/
|
|
|
- public void closePipelineIfNoOpenContainers(Pipeline pipeline)
|
|
|
+ private void closePipelineIfNoOpenContainers(Pipeline pipeline)
|
|
|
throws IOException {
|
|
|
if (pipeline.getLifeCycleState() != LifeCycleState.CLOSING) {
|
|
|
return;
|
|
@@ -354,7 +334,7 @@ public class PipelineSelector {
|
|
|
* Close a given pipeline.
|
|
|
*/
|
|
|
private void closePipeline(Pipeline pipeline) throws IOException {
|
|
|
- PipelineManager manager = getPipelineManager(pipeline.getType());
|
|
|
+ PipelineManager manager = pipelineManagerMap.get(pipeline.getType());
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
LOG.debug("Closing pipeline. pipelineID: {}", pipeline.getId());
|
|
|
HashSet<ContainerID> containers =
|
|
@@ -367,7 +347,7 @@ public class PipelineSelector {
|
|
|
* Add to a given pipeline.
|
|
|
*/
|
|
|
private void addOpenPipeline(Pipeline pipeline) {
|
|
|
- PipelineManager manager = getPipelineManager(pipeline.getType());
|
|
|
+ PipelineManager manager = pipelineManagerMap.get(pipeline.getType());
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
LOG.debug("Adding Open pipeline. pipelineID: {}", pipeline.getId());
|
|
|
manager.addOpenPipeline(pipeline);
|
|
@@ -381,7 +361,7 @@ public class PipelineSelector {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public Set<PipelineID> getPipelineId(UUID dnId) {
|
|
|
+ public Set<PipelineID> getPipelineByDnID(UUID dnId) {
|
|
|
return node2PipelineMap.getPipelines(dnId);
|
|
|
}
|
|
|
|
|
@@ -400,6 +380,9 @@ public class PipelineSelector {
|
|
|
pipelineMap.put(pipeline.getId(), pipeline);
|
|
|
pipeline2ContainerMap.put(pipeline.getId(), new HashSet<>());
|
|
|
node2PipelineMap.addPipeline(pipeline);
|
|
|
+ // reset the datanodes in the pipeline
|
|
|
+ // they will be reset on
|
|
|
+ pipeline.resetPipeline();
|
|
|
break;
|
|
|
case CLOSED:
|
|
|
// if the pipeline is in closed state, nothing to do.
|
|
@@ -409,6 +392,36 @@ public class PipelineSelector {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ public void handleStaleNode(DatanodeDetails dn) {
|
|
|
+ Set<PipelineID> pipelineIDs = getPipelineByDnID(dn.getUuid());
|
|
|
+ for (PipelineID id : pipelineIDs) {
|
|
|
+ LOG.info("closing pipeline {}.", id);
|
|
|
+ eventPublisher.fireEvent(SCMEvents.PIPELINE_CLOSE, id);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ void processPipelineReport(DatanodeDetails dn,
|
|
|
+ PipelineReportsProto pipelineReport) {
|
|
|
+ Set<PipelineID> reportedPipelines = new HashSet<>();
|
|
|
+ pipelineReport.getPipelineReportList().
|
|
|
+ forEach(p ->
|
|
|
+ reportedPipelines.add(
|
|
|
+ processPipelineReport(p.getPipelineID(), dn)));
|
|
|
+
|
|
|
+ //TODO: handle missing pipelines and new pipelines later
|
|
|
+ }
|
|
|
+
|
|
|
+ private PipelineID processPipelineReport(
|
|
|
+ HddsProtos.PipelineID id, DatanodeDetails dn) {
|
|
|
+ PipelineID pipelineID = PipelineID.getFromProtobuf(id);
|
|
|
+ Pipeline pipeline = pipelineMap.get(pipelineID);
|
|
|
+ if (pipeline != null) {
|
|
|
+ pipelineManagerMap.get(pipeline.getType())
|
|
|
+ .processPipelineReport(pipeline, dn);
|
|
|
+ }
|
|
|
+ return pipelineID;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Update the Pipeline State to the next state.
|
|
|
*
|