|
@@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
|
|
|
+import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
|
|
|
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
|
|
.ContainerPlacementPolicy;
|
|
|
import org.apache.hadoop.hdds.scm.container.placement.algorithms
|
|
@@ -184,13 +185,13 @@ public class PipelineSelector {
|
|
|
*/
|
|
|
public static Pipeline newPipelineFromNodes(
|
|
|
List<DatanodeDetails> nodes, ReplicationType replicationType,
|
|
|
- ReplicationFactor replicationFactor, String name) {
|
|
|
+ ReplicationFactor replicationFactor, PipelineID id) {
|
|
|
Preconditions.checkNotNull(nodes);
|
|
|
Preconditions.checkArgument(nodes.size() > 0);
|
|
|
String leaderId = nodes.get(0).getUuidString();
|
|
|
// A new pipeline always starts in allocated state
|
|
|
Pipeline pipeline = new Pipeline(leaderId, LifeCycleState.ALLOCATED,
|
|
|
- replicationType, replicationFactor, name);
|
|
|
+ replicationType, replicationFactor, id);
|
|
|
for (DatanodeDetails node : nodes) {
|
|
|
pipeline.addMember(node);
|
|
|
}
|
|
@@ -304,16 +305,16 @@ public class PipelineSelector {
|
|
|
* This function to return pipeline for given pipeline name and replication
|
|
|
* type.
|
|
|
*/
|
|
|
- public Pipeline getPipeline(String pipelineName,
|
|
|
+ public Pipeline getPipeline(PipelineID pipelineID,
|
|
|
ReplicationType replicationType) throws IOException {
|
|
|
- if (pipelineName == null) {
|
|
|
+ if (pipelineID == null) {
|
|
|
return null;
|
|
|
}
|
|
|
PipelineManager manager = getPipelineManager(replicationType);
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
LOG.debug("Getting replication pipeline forReplicationType {} :" +
|
|
|
- " pipelineName:{}", replicationType, pipelineName);
|
|
|
- return manager.getPipeline(pipelineName);
|
|
|
+ " pipelineName:{}", replicationType, pipelineID);
|
|
|
+ return manager.getPipeline(pipelineID);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -322,7 +323,7 @@ public class PipelineSelector {
|
|
|
public void finalizePipeline(Pipeline pipeline) throws IOException {
|
|
|
PipelineManager manager = getPipelineManager(pipeline.getType());
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
- LOG.debug("Finalizing pipeline. pipelineID: {}", pipeline.getPipelineName());
|
|
|
+ LOG.debug("Finalizing pipeline. pipelineID: {}", pipeline.getId());
|
|
|
// Remove the pipeline from active allocation
|
|
|
manager.finalizePipeline(pipeline);
|
|
|
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.FINALIZE);
|
|
@@ -337,10 +338,10 @@ public class PipelineSelector {
|
|
|
return;
|
|
|
}
|
|
|
NavigableSet<ContainerID> containerIDS = containerStateManager
|
|
|
- .getMatchingContainerIDsByPipeline(pipeline.getPipelineName());
|
|
|
+ .getMatchingContainerIDsByPipeline(pipeline.getId());
|
|
|
if (containerIDS.size() == 0) {
|
|
|
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CLOSE);
|
|
|
- LOG.info("Closing pipeline. pipelineID: {}", pipeline.getPipelineName());
|
|
|
+ LOG.info("Closing pipeline. pipelineID: {}", pipeline.getId());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -350,10 +351,10 @@ public class PipelineSelector {
|
|
|
private void closePipeline(Pipeline pipeline) {
|
|
|
PipelineManager manager = getPipelineManager(pipeline.getType());
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
- LOG.debug("Closing pipeline. pipelineID: {}", pipeline.getPipelineName());
|
|
|
+ LOG.debug("Closing pipeline. pipelineID: {}", pipeline.getId());
|
|
|
NavigableSet<ContainerID> containers =
|
|
|
containerStateManager
|
|
|
- .getMatchingContainerIDsByPipeline(pipeline.getPipelineName());
|
|
|
+ .getMatchingContainerIDsByPipeline(pipeline.getId());
|
|
|
Preconditions.checkArgument(containers.size() == 0);
|
|
|
manager.closePipeline(pipeline);
|
|
|
}
|
|
@@ -361,7 +362,7 @@ public class PipelineSelector {
|
|
|
private void closeContainersByPipeline(Pipeline pipeline) {
|
|
|
NavigableSet<ContainerID> containers =
|
|
|
containerStateManager
|
|
|
- .getMatchingContainerIDsByPipeline(pipeline.getPipelineName());
|
|
|
+ .getMatchingContainerIDsByPipeline(pipeline.getId());
|
|
|
for (ContainerID id : containers) {
|
|
|
eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, id);
|
|
|
}
|
|
@@ -372,7 +373,7 @@ public class PipelineSelector {
|
|
|
*/
|
|
|
|
|
|
public List<DatanodeDetails> getDatanodes(ReplicationType replicationType,
|
|
|
- String pipelineID) throws IOException {
|
|
|
+ PipelineID pipelineID) throws IOException {
|
|
|
PipelineManager manager = getPipelineManager(replicationType);
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
LOG.debug("Getting data nodes from pipeline : {}", pipelineID);
|
|
@@ -383,7 +384,7 @@ public class PipelineSelector {
|
|
|
* Update the datanodes in the list of the pipeline.
|
|
|
*/
|
|
|
|
|
|
- public void updateDatanodes(ReplicationType replicationType, String
|
|
|
+ public void updateDatanodes(ReplicationType replicationType, PipelineID
|
|
|
pipelineID, List<DatanodeDetails> newDatanodes) throws IOException {
|
|
|
PipelineManager manager = getPipelineManager(replicationType);
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
@@ -423,7 +424,7 @@ public class PipelineSelector {
|
|
|
String error = String.format("Failed to update pipeline state %s, " +
|
|
|
"reason: invalid state transition from state: %s upon " +
|
|
|
"event: %s.",
|
|
|
- pipeline.getPipelineName(), pipeline.getLifeCycleState(), event);
|
|
|
+ pipeline.getId(), pipeline.getLifeCycleState(), event);
|
|
|
LOG.error(error);
|
|
|
throw new SCMException(error, FAILED_TO_CHANGE_PIPELINE_STATE);
|
|
|
}
|