|
@@ -21,7 +21,6 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
|
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
|
|
@@ -39,30 +38,34 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
|
|
-import org.apache.hadoop.ozone.common.statemachine
|
|
|
- .InvalidStateTransitionException;
|
|
|
-import org.apache.hadoop.ozone.common.statemachine.StateMachine;
|
|
|
+import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
import org.apache.hadoop.ozone.lease.Lease;
|
|
|
import org.apache.hadoop.ozone.lease.LeaseException;
|
|
|
import org.apache.hadoop.ozone.lease.LeaseManager;
|
|
|
+import org.apache.hadoop.utils.MetadataStore;
|
|
|
+import org.apache.hadoop.utils.MetadataStoreBuilder;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.lang.reflect.Constructor;
|
|
|
import java.lang.reflect.InvocationTargetException;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
-import java.util.NavigableSet;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.Set;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.UUID;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
-import java.util.stream.Collectors;
|
|
|
|
|
|
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
|
|
|
.FAILED_TO_CHANGE_PIPELINE_STATE;
|
|
|
+import static org.apache.hadoop.hdds.server
|
|
|
+ .ServerUtils.getOzoneMetaDirPath;
|
|
|
+import static org.apache.hadoop.ozone
|
|
|
+ .OzoneConsts.SCM_PIPELINE_DB;
|
|
|
|
|
|
/**
|
|
|
* Sends the request to the right pipeline manager.
|
|
@@ -73,16 +76,16 @@ public class PipelineSelector {
|
|
|
private final ContainerPlacementPolicy placementPolicy;
|
|
|
private final NodeManager nodeManager;
|
|
|
private final Configuration conf;
|
|
|
- private final ContainerStateManager containerStateManager;
|
|
|
private final EventPublisher eventPublisher;
|
|
|
private final RatisManagerImpl ratisManager;
|
|
|
private final StandaloneManagerImpl standaloneManager;
|
|
|
private final long containerSize;
|
|
|
+ private final MetadataStore pipelineStore;
|
|
|
+ private final PipelineStateManager stateManager;
|
|
|
private final Node2PipelineMap node2PipelineMap;
|
|
|
+ private final Map<PipelineID, HashSet<ContainerID>> pipeline2ContainerMap;
|
|
|
private final Map<PipelineID, Pipeline> pipelineMap;
|
|
|
private final LeaseManager<Pipeline> pipelineLeaseManager;
|
|
|
- private final StateMachine<LifeCycleState,
|
|
|
- HddsProtos.LifeCycleEvent> stateMachine;
|
|
|
|
|
|
/**
|
|
|
* Constructs a pipeline Selector.
|
|
@@ -90,9 +93,8 @@ public class PipelineSelector {
|
|
|
* @param nodeManager - node manager
|
|
|
* @param conf - Ozone Config
|
|
|
*/
|
|
|
- public PipelineSelector(NodeManager nodeManager,
|
|
|
- ContainerStateManager containerStateManager, Configuration conf,
|
|
|
- EventPublisher eventPublisher) {
|
|
|
+ public PipelineSelector(NodeManager nodeManager, Configuration conf,
|
|
|
+ EventPublisher eventPublisher, int cacheSizeMB) throws IOException {
|
|
|
this.nodeManager = nodeManager;
|
|
|
this.conf = conf;
|
|
|
this.eventPublisher = eventPublisher;
|
|
@@ -105,79 +107,66 @@ public class PipelineSelector {
|
|
|
pipelineMap = new ConcurrentHashMap<>();
|
|
|
this.standaloneManager =
|
|
|
new StandaloneManagerImpl(this.nodeManager, placementPolicy,
|
|
|
- containerSize, node2PipelineMap, pipelineMap);
|
|
|
+ containerSize);
|
|
|
this.ratisManager =
|
|
|
new RatisManagerImpl(this.nodeManager, placementPolicy, containerSize,
|
|
|
- conf, node2PipelineMap, pipelineMap);
|
|
|
- // Initialize the container state machine.
|
|
|
- Set<HddsProtos.LifeCycleState> finalStates = new HashSet();
|
|
|
+ conf);
|
|
|
long pipelineCreationLeaseTimeout = conf.getTimeDuration(
|
|
|
ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT,
|
|
|
ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT_DEFAULT,
|
|
|
TimeUnit.MILLISECONDS);
|
|
|
- this.containerStateManager = containerStateManager;
|
|
|
pipelineLeaseManager = new LeaseManager<>("PipelineCreation",
|
|
|
pipelineCreationLeaseTimeout);
|
|
|
pipelineLeaseManager.start();
|
|
|
|
|
|
- // These are the steady states of a container.
|
|
|
- finalStates.add(HddsProtos.LifeCycleState.OPEN);
|
|
|
- finalStates.add(HddsProtos.LifeCycleState.CLOSED);
|
|
|
+ stateManager = new PipelineStateManager();
|
|
|
+ pipeline2ContainerMap = new HashMap<>();
|
|
|
|
|
|
- this.stateMachine = new StateMachine<>(HddsProtos.LifeCycleState.ALLOCATED,
|
|
|
- finalStates);
|
|
|
- initializeStateMachine();
|
|
|
+ // Write the container name to pipeline mapping.
|
|
|
+ File metaDir = getOzoneMetaDirPath(conf);
|
|
|
+ File containerDBPath = new File(metaDir, SCM_PIPELINE_DB);
|
|
|
+ pipelineStore = MetadataStoreBuilder.newBuilder()
|
|
|
+ .setConf(conf)
|
|
|
+ .setDbFile(containerDBPath)
|
|
|
+ .setCacheSize(cacheSizeMB * OzoneConsts.MB)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ reloadExistingPipelines();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Event and State Transition Mapping.
|
|
|
- *
|
|
|
- * State: ALLOCATED ---------------> CREATING
|
|
|
- * Event: CREATE
|
|
|
- *
|
|
|
- * State: CREATING ---------------> OPEN
|
|
|
- * Event: CREATED
|
|
|
- *
|
|
|
- * State: OPEN ---------------> CLOSING
|
|
|
- * Event: FINALIZE
|
|
|
- *
|
|
|
- * State: CLOSING ---------------> CLOSED
|
|
|
- * Event: CLOSE
|
|
|
- *
|
|
|
- * State: CREATING ---------------> CLOSED
|
|
|
- * Event: TIMEOUT
|
|
|
- *
|
|
|
- *
|
|
|
- * Container State Flow:
|
|
|
- *
|
|
|
- * [ALLOCATED]---->[CREATING]------>[OPEN]-------->[CLOSING]
|
|
|
- * (CREATE) | (CREATED) (FINALIZE) |
|
|
|
- * | |
|
|
|
- * | |
|
|
|
- * |(TIMEOUT) |(CLOSE)
|
|
|
- * | |
|
|
|
- * +--------> [CLOSED] <--------+
|
|
|
- */
|
|
|
- private void initializeStateMachine() {
|
|
|
- stateMachine.addTransition(HddsProtos.LifeCycleState.ALLOCATED,
|
|
|
- HddsProtos.LifeCycleState.CREATING,
|
|
|
- HddsProtos.LifeCycleEvent.CREATE);
|
|
|
-
|
|
|
- stateMachine.addTransition(HddsProtos.LifeCycleState.CREATING,
|
|
|
- HddsProtos.LifeCycleState.OPEN,
|
|
|
- HddsProtos.LifeCycleEvent.CREATED);
|
|
|
-
|
|
|
- stateMachine.addTransition(HddsProtos.LifeCycleState.OPEN,
|
|
|
- HddsProtos.LifeCycleState.CLOSING,
|
|
|
- HddsProtos.LifeCycleEvent.FINALIZE);
|
|
|
-
|
|
|
- stateMachine.addTransition(HddsProtos.LifeCycleState.CLOSING,
|
|
|
- HddsProtos.LifeCycleState.CLOSED,
|
|
|
- HddsProtos.LifeCycleEvent.CLOSE);
|
|
|
-
|
|
|
- stateMachine.addTransition(HddsProtos.LifeCycleState.CREATING,
|
|
|
- HddsProtos.LifeCycleState.CLOSED,
|
|
|
- HddsProtos.LifeCycleEvent.TIMEOUT);
|
|
|
+ private void reloadExistingPipelines() throws IOException {
|
|
|
+ if (pipelineStore.isEmpty()) {
|
|
|
+ // Nothing to do just return
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ List<Map.Entry<byte[], byte[]>> range =
|
|
|
+ pipelineStore.getSequentialRangeKVs(null, Integer.MAX_VALUE, null);
|
|
|
+
|
|
|
+ // Transform the values into the pipelines.
|
|
|
+ // TODO: filter by pipeline state
|
|
|
+ for (Map.Entry<byte[], byte[]> entry : range) {
|
|
|
+ Pipeline pipeline = Pipeline.getFromProtoBuf(
|
|
|
+ HddsProtos.Pipeline.PARSER.parseFrom(entry.getValue()));
|
|
|
+ Preconditions.checkNotNull(pipeline);
|
|
|
+ addExistingPipeline(pipeline);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public Set<ContainerID> getOpenContainerIDsByPipeline(PipelineID pipelineID) {
|
|
|
+ return pipeline2ContainerMap.get(pipelineID);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void addContainerToPipeline(PipelineID pipelineID, long containerID) {
|
|
|
+ pipeline2ContainerMap.get(pipelineID)
|
|
|
+ .add(ContainerID.valueof(containerID));
|
|
|
+ }
|
|
|
+
|
|
|
+ public void removeContainerFromPipeline(PipelineID pipelineID,
|
|
|
+ long containerID) throws IOException {
|
|
|
+ pipeline2ContainerMap.get(pipelineID)
|
|
|
+ .remove(ContainerID.valueof(containerID));
|
|
|
+ closePipelineIfNoOpenContainers(pipelineMap.get(pipelineID));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -294,8 +283,14 @@ public class PipelineSelector {
|
|
|
manager.createPipeline(replicationFactor, replicationType);
|
|
|
if (pipeline == null) {
|
|
|
// try to return a pipeline from already allocated pipelines
|
|
|
- pipeline = manager.getPipeline(replicationFactor, replicationType);
|
|
|
+ PipelineID pipelineId =
|
|
|
+ manager.getPipeline(replicationFactor, replicationType);
|
|
|
+ pipeline = pipelineMap.get(pipelineId);
|
|
|
+ Preconditions.checkArgument(pipeline.getLifeCycleState() ==
|
|
|
+ LifeCycleState.OPEN);
|
|
|
} else {
|
|
|
+ pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
|
|
|
+ pipeline.getProtobufMessage().toByteArray());
|
|
|
// if a new pipeline is created, initialize its state machine
|
|
|
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CREATE);
|
|
|
|
|
@@ -343,8 +338,8 @@ public class PipelineSelector {
|
|
|
if (pipeline.getLifeCycleState() != LifeCycleState.CLOSING) {
|
|
|
return;
|
|
|
}
|
|
|
- NavigableSet<ContainerID> containerIDS = containerStateManager
|
|
|
- .getMatchingContainerIDsByPipeline(pipeline.getId());
|
|
|
+ HashSet<ContainerID> containerIDS =
|
|
|
+ pipeline2ContainerMap.get(pipeline.getId());
|
|
|
if (containerIDS.size() == 0) {
|
|
|
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CLOSE);
|
|
|
LOG.info("Closing pipeline. pipelineID: {}", pipeline.getId());
|
|
@@ -358,56 +353,58 @@ public class PipelineSelector {
|
|
|
PipelineManager manager = getPipelineManager(pipeline.getType());
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
LOG.debug("Closing pipeline. pipelineID: {}", pipeline.getId());
|
|
|
- NavigableSet<ContainerID> containers =
|
|
|
- containerStateManager
|
|
|
- .getMatchingContainerIDsByPipeline(pipeline.getId());
|
|
|
+ HashSet<ContainerID> containers =
|
|
|
+ pipeline2ContainerMap.get(pipeline.getId());
|
|
|
Preconditions.checkArgument(containers.size() == 0);
|
|
|
manager.closePipeline(pipeline);
|
|
|
}
|
|
|
|
|
|
- private void closeContainersByPipeline(Pipeline pipeline) {
|
|
|
- NavigableSet<ContainerID> containers =
|
|
|
- containerStateManager
|
|
|
- .getMatchingContainerIDsByPipeline(pipeline.getId());
|
|
|
- for (ContainerID id : containers) {
|
|
|
- eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, id);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
- * list members in the pipeline .
|
|
|
+ * Add to a given pipeline.
|
|
|
*/
|
|
|
-
|
|
|
- public List<DatanodeDetails> getDatanodes(ReplicationType replicationType,
|
|
|
- PipelineID pipelineID) throws IOException {
|
|
|
- PipelineManager manager = getPipelineManager(replicationType);
|
|
|
- Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
- LOG.debug("Getting data nodes from pipeline : {}", pipelineID);
|
|
|
- return manager.getMembers(pipelineID);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Update the datanodes in the list of the pipeline.
|
|
|
- */
|
|
|
-
|
|
|
- public void updateDatanodes(ReplicationType replicationType, PipelineID
|
|
|
- pipelineID, List<DatanodeDetails> newDatanodes) throws IOException {
|
|
|
- PipelineManager manager = getPipelineManager(replicationType);
|
|
|
+ private void addOpenPipeline(Pipeline pipeline) {
|
|
|
+ PipelineManager manager = getPipelineManager(pipeline.getType());
|
|
|
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
|
|
|
- LOG.debug("Updating pipeline: {} with new nodes:{}", pipelineID,
|
|
|
- newDatanodes.stream().map(DatanodeDetails::toString)
|
|
|
- .collect(Collectors.joining(",")));
|
|
|
- manager.updatePipeline(pipelineID, newDatanodes);
|
|
|
+ LOG.debug("Adding Open pipeline. pipelineID: {}", pipeline.getId());
|
|
|
+ manager.addOpenPipeline(pipeline);
|
|
|
}
|
|
|
|
|
|
- public Node2PipelineMap getNode2PipelineMap() {
|
|
|
- return node2PipelineMap;
|
|
|
+ private void closeContainersByPipeline(Pipeline pipeline) {
|
|
|
+ HashSet<ContainerID> containers =
|
|
|
+ pipeline2ContainerMap.get(pipeline.getId());
|
|
|
+ for (ContainerID id : containers) {
|
|
|
+ eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, id);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public Set<PipelineID> getPipelineId(UUID dnId) {
|
|
|
return node2PipelineMap.getPipelines(dnId);
|
|
|
}
|
|
|
|
|
|
+ private void addExistingPipeline(Pipeline pipeline) throws IOException {
|
|
|
+ LifeCycleState state = pipeline.getLifeCycleState();
|
|
|
+ switch (state) {
|
|
|
+ case ALLOCATED:
|
|
|
+ // a pipeline in allocated state is only present in SCM and does not exist
|
|
|
+ // on datanode, on SCM restart, this pipeline can be ignored.
|
|
|
+ break;
|
|
|
+ case CREATING:
|
|
|
+ case OPEN:
|
|
|
+ case CLOSING:
|
|
|
+ //TODO: process pipeline report and move pipeline to active queue
|
|
|
+ // when all the nodes have reported.
|
|
|
+ pipelineMap.put(pipeline.getId(), pipeline);
|
|
|
+ pipeline2ContainerMap.put(pipeline.getId(), new HashSet<>());
|
|
|
+ node2PipelineMap.addPipeline(pipeline);
|
|
|
+ break;
|
|
|
+ case CLOSED:
|
|
|
+ // if the pipeline is in closed state, nothing to do.
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ throw new IOException("invalid pipeline state:" + state);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Update the Pipeline State to the next state.
|
|
|
*
|
|
@@ -417,24 +414,12 @@ public class PipelineSelector {
|
|
|
*/
|
|
|
public void updatePipelineState(Pipeline pipeline,
|
|
|
HddsProtos.LifeCycleEvent event) throws IOException {
|
|
|
- HddsProtos.LifeCycleState newState;
|
|
|
- try {
|
|
|
- newState = stateMachine.getNextState(pipeline.getLifeCycleState(), event);
|
|
|
- } catch (InvalidStateTransitionException ex) {
|
|
|
- String error = String.format("Failed to update pipeline state %s, " +
|
|
|
- "reason: invalid state transition from state: %s upon " +
|
|
|
- "event: %s.",
|
|
|
- pipeline.getId(), pipeline.getLifeCycleState(), event);
|
|
|
- LOG.error(error);
|
|
|
- throw new SCMException(error, FAILED_TO_CHANGE_PIPELINE_STATE);
|
|
|
- }
|
|
|
-
|
|
|
- // This is a post condition after executing getNextState.
|
|
|
- Preconditions.checkNotNull(newState);
|
|
|
- Preconditions.checkNotNull(pipeline);
|
|
|
try {
|
|
|
switch (event) {
|
|
|
case CREATE:
|
|
|
+ pipelineMap.put(pipeline.getId(), pipeline);
|
|
|
+ pipeline2ContainerMap.put(pipeline.getId(), new HashSet<>());
|
|
|
+ node2PipelineMap.addPipeline(pipeline);
|
|
|
// Acquire lease on pipeline
|
|
|
Lease<Pipeline> pipelineLease = pipelineLeaseManager.acquire(pipeline);
|
|
|
// Register callback to be executed in case of timeout
|
|
@@ -446,6 +431,7 @@ public class PipelineSelector {
|
|
|
case CREATED:
|
|
|
// Release the lease on pipeline
|
|
|
pipelineLeaseManager.release(pipeline);
|
|
|
+ addOpenPipeline(pipeline);
|
|
|
break;
|
|
|
|
|
|
case FINALIZE:
|
|
@@ -455,21 +441,30 @@ public class PipelineSelector {
|
|
|
case CLOSE:
|
|
|
case TIMEOUT:
|
|
|
closePipeline(pipeline);
|
|
|
+ pipeline2ContainerMap.remove(pipeline.getId());
|
|
|
+ node2PipelineMap.removePipeline(pipeline);
|
|
|
+ pipelineMap.remove(pipeline.getId());
|
|
|
break;
|
|
|
default:
|
|
|
throw new SCMException("Unsupported pipeline LifeCycleEvent.",
|
|
|
FAILED_TO_CHANGE_PIPELINE_STATE);
|
|
|
}
|
|
|
|
|
|
- pipeline.setLifeCycleState(newState);
|
|
|
+ stateManager.updatePipelineState(pipeline, event);
|
|
|
+ pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
|
|
|
+ pipeline.getProtobufMessage().toByteArray());
|
|
|
} catch (LeaseException e) {
|
|
|
throw new IOException("Lease Exception.", e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public void shutdown() {
|
|
|
+ public void shutdown() throws IOException {
|
|
|
if (pipelineLeaseManager != null) {
|
|
|
pipelineLeaseManager.shutdown();
|
|
|
}
|
|
|
+
|
|
|
+ if (pipelineStore != null) {
|
|
|
+ pipelineStore.close();
|
|
|
+ }
|
|
|
}
|
|
|
}
|