|
@@ -26,13 +26,15 @@ import org.apache.hadoop.hdds.scm.ScmUtils;
|
|
|
import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
|
|
-import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
|
|
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
|
import org.apache.hadoop.hdds.scm.node.NodeManager;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
|
|
|
import org.apache.hadoop.hdds.server.events.EventHandler;
|
|
|
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
|
|
import org.apache.hadoop.metrics2.util.MBeans;
|
|
@@ -70,6 +72,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
// Currently only user of the block service is Ozone, CBlock manages blocks
|
|
|
// by itself and does not rely on the Block service offered by SCM.
|
|
|
|
|
|
+ private final PipelineManager pipelineManager;
|
|
|
private final ContainerManager containerManager;
|
|
|
|
|
|
private final long containerSize;
|
|
@@ -87,14 +90,16 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
*
|
|
|
* @param conf - configuration.
|
|
|
* @param nodeManager - node manager.
|
|
|
+ * @param pipelineManager - pipeline manager.
|
|
|
* @param containerManager - container manager.
|
|
|
* @param eventPublisher - event publisher.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
public BlockManagerImpl(final Configuration conf,
|
|
|
- final NodeManager nodeManager, final ContainerManager containerManager,
|
|
|
- EventPublisher eventPublisher)
|
|
|
+ final NodeManager nodeManager, final PipelineManager pipelineManager,
|
|
|
+ final ContainerManager containerManager, EventPublisher eventPublisher)
|
|
|
throws IOException {
|
|
|
+ this.pipelineManager = pipelineManager;
|
|
|
this.containerManager = containerManager;
|
|
|
|
|
|
this.containerSize = (long)conf.getStorageSize(
|
|
@@ -155,16 +160,15 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
private synchronized void preAllocateContainers(int count,
|
|
|
- ReplicationType type, ReplicationFactor factor, String owner)
|
|
|
- throws IOException {
|
|
|
+ ReplicationType type, ReplicationFactor factor, String owner) {
|
|
|
for (int i = 0; i < count; i++) {
|
|
|
- ContainerWithPipeline containerWithPipeline;
|
|
|
+ ContainerInfo containerInfo;
|
|
|
try {
|
|
|
// TODO: Fix this later when Ratis is made the Default.
|
|
|
- containerWithPipeline = containerManager.allocateContainer(
|
|
|
+ containerInfo = containerManager.allocateContainer(
|
|
|
type, factor, owner);
|
|
|
|
|
|
- if (containerWithPipeline == null) {
|
|
|
+ if (containerInfo == null) {
|
|
|
LOG.warn("Unable to allocate container.");
|
|
|
}
|
|
|
} catch (IOException ex) {
|
|
@@ -206,11 +210,11 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
use different kind of policies.
|
|
|
*/
|
|
|
|
|
|
- ContainerWithPipeline containerWithPipeline;
|
|
|
+ ContainerInfo containerInfo;
|
|
|
|
|
|
// look for OPEN containers that match the criteria.
|
|
|
- containerWithPipeline = containerManager
|
|
|
- .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
+ containerInfo = containerManager
|
|
|
+ .getMatchingContainer(size, owner, type, factor,
|
|
|
HddsProtos.LifeCycleState.OPEN);
|
|
|
|
|
|
// We did not find OPEN Containers. This generally means
|
|
@@ -221,27 +225,27 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
// Even though we have already checked the containers in OPEN
|
|
|
// state, we have to check again as we only hold a read lock.
|
|
|
// Some other thread might have pre-allocated container in meantime.
|
|
|
- if (containerWithPipeline == null) {
|
|
|
+ if (containerInfo == null) {
|
|
|
synchronized (this) {
|
|
|
if (!containerManager.getContainers(HddsProtos.LifeCycleState.OPEN)
|
|
|
.isEmpty()) {
|
|
|
- containerWithPipeline = containerManager
|
|
|
- .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
+ containerInfo = containerManager
|
|
|
+ .getMatchingContainer(size, owner, type, factor,
|
|
|
HddsProtos.LifeCycleState.OPEN);
|
|
|
}
|
|
|
|
|
|
- if (containerWithPipeline == null) {
|
|
|
+ if (containerInfo == null) {
|
|
|
preAllocateContainers(containerProvisionBatchSize, type, factor,
|
|
|
owner);
|
|
|
- containerWithPipeline = containerManager
|
|
|
- .getMatchingContainerWithPipeline(size, owner, type, factor,
|
|
|
+ containerInfo = containerManager
|
|
|
+ .getMatchingContainer(size, owner, type, factor,
|
|
|
HddsProtos.LifeCycleState.OPEN);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- if (containerWithPipeline != null) {
|
|
|
- return newBlock(containerWithPipeline, HddsProtos.LifeCycleState.OPEN);
|
|
|
+ if (containerInfo != null) {
|
|
|
+ return newBlock(containerInfo);
|
|
|
}
|
|
|
|
|
|
// we have tried all strategies we know and but somehow we are not able
|
|
@@ -255,29 +259,26 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|
|
/**
|
|
|
* newBlock - returns a new block assigned to a container.
|
|
|
*
|
|
|
- * @param containerWithPipeline - Container Info.
|
|
|
- * @param state - Current state of the container.
|
|
|
+ * @param containerInfo - Container Info.
|
|
|
* @return AllocatedBlock
|
|
|
*/
|
|
|
- private AllocatedBlock newBlock(ContainerWithPipeline containerWithPipeline,
|
|
|
- HddsProtos.LifeCycleState state) throws IOException {
|
|
|
- ContainerInfo containerInfo = containerWithPipeline.getContainerInfo();
|
|
|
- if (containerWithPipeline.getPipeline().getNodes().size() == 0) {
|
|
|
- LOG.error("Pipeline Machine count is zero.");
|
|
|
+ private AllocatedBlock newBlock(ContainerInfo containerInfo) {
|
|
|
+ try {
|
|
|
+ final Pipeline pipeline = pipelineManager
|
|
|
+ .getPipeline(containerInfo.getPipelineID());
|
|
|
+ // TODO : Revisit this local ID allocation when HA is added.
|
|
|
+ long localID = UniqueId.next();
|
|
|
+ long containerID = containerInfo.getContainerID();
|
|
|
+ AllocatedBlock.Builder abb = new AllocatedBlock.Builder()
|
|
|
+ .setContainerBlockID(new ContainerBlockID(containerID, localID))
|
|
|
+ .setPipeline(pipeline);
|
|
|
+ LOG.trace("New block allocated : {} Container ID: {}", localID,
|
|
|
+ containerID);
|
|
|
+ return abb.build();
|
|
|
+ } catch (PipelineNotFoundException ex) {
|
|
|
+ LOG.error("Pipeline Machine count is zero.", ex);
|
|
|
return null;
|
|
|
}
|
|
|
-
|
|
|
- // TODO : Revisit this local ID allocation when HA is added.
|
|
|
- long localID = UniqueId.next();
|
|
|
- long containerID = containerInfo.getContainerID();
|
|
|
-
|
|
|
- AllocatedBlock.Builder abb =
|
|
|
- new AllocatedBlock.Builder()
|
|
|
- .setContainerBlockID(new ContainerBlockID(containerID, localID))
|
|
|
- .setPipeline(containerWithPipeline.getPipeline());
|
|
|
- LOG.trace("New block allocated : {} Container ID: {}", localID,
|
|
|
- containerID);
|
|
|
- return abb.build();
|
|
|
}
|
|
|
|
|
|
/**
|