You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by na...@apache.org on 2020/03/31 17:38:59 UTC
[hadoop-ozone] branch master updated: HDDS-3156 update
allocateContainer to remove additional createPipeline step. (#663)
This is an automated email from the ASF dual-hosted git repository.
nanda pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hadoop-ozone.git
The following commit(s) were added to refs/heads/master by this push:
new f2343f1 HDDS-3156 update allocateContainer to remove additional createPipeline step. (#663)
f2343f1 is described below
commit f2343f176a52254a6fb595f02c6b097a106e5d06
Author: Li Cheng <bl...@gmail.com>
AuthorDate: Wed Apr 1 01:38:52 2020 +0800
HDDS-3156 update allocateContainer to remove additional createPipeline step. (#663)
---
.../hdds/scm/container/ContainerStateManager.java | 42 ++++++++++++++--------
1 file changed, 28 insertions(+), 14 deletions(-)
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
index 526e4b3..36259c4 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerStateManager.java
@@ -118,6 +118,7 @@ public class ContainerStateManager {
HddsProtos.LifeCycleEvent> stateMachine;
private final long containerSize;
+ private final boolean autoCreateRatisOne;
private final ConcurrentHashMap<ContainerState, ContainerID> lastUsedMap;
private final ContainerStateMap containers;
private final AtomicLong containerCount;
@@ -149,6 +150,9 @@ public class ContainerStateManager {
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
StorageUnit.BYTES);
+ this.autoCreateRatisOne = configuration.getBoolean(
+ ScmConfigKeys.OZONE_SCM_PIPELINE_AUTO_CREATE_FACTOR_ONE,
+ ScmConfigKeys.OZONE_SCM_PIPELINE_AUTO_CREATE_FACTOR_ONE_DEFAULT);
this.lastUsedMap = new ConcurrentHashMap<>();
this.containerCount = new AtomicLong(0);
@@ -247,23 +251,33 @@ public class ContainerStateManager {
final HddsProtos.ReplicationType type,
final HddsProtos.ReplicationFactor replicationFactor, final String owner)
throws IOException {
-
+ final List<Pipeline> pipelines = pipelineManager
+ .getPipelines(type, replicationFactor, Pipeline.PipelineState.OPEN);
Pipeline pipeline;
- try {
- // TODO: #CLUTIL remove creation logic when all replication types and
- // factors are handled by pipeline creator job.
- pipeline = pipelineManager.createPipeline(type, replicationFactor);
- pipelineManager.waitPipelineReady(pipeline.getId(), 0);
- } catch (IOException e) {
- final List<Pipeline> pipelines = pipelineManager
- .getPipelines(type, replicationFactor, Pipeline.PipelineState.OPEN);
- if (pipelines.isEmpty()) {
- throw new IOException("Could not allocate container. Cannot get any" +
- " matching pipeline for Type:" + type +
- ", Factor:" + replicationFactor + ", State:PipelineState.OPEN");
- }
+
+ boolean bgCreateOne = (type == ReplicationType.RATIS) && replicationFactor
+ == ReplicationFactor.ONE && autoCreateRatisOne;
+ boolean bgCreateThree = (type == ReplicationType.RATIS) && replicationFactor
+ == ReplicationFactor.THREE;
+
+ if (!pipelines.isEmpty() && (bgCreateOne || bgCreateThree)) {
+ // let background create Ratis pipelines.
pipeline = pipelines.get((int) containerCount.get() % pipelines.size());
+ } else {
+ try {
+ pipeline = pipelineManager.createPipeline(type, replicationFactor);
+ pipelineManager.waitPipelineReady(pipeline.getId(), 0);
+ } catch (IOException e) {
+
+ if (pipelines.isEmpty()) {
+ throw new IOException("Could not allocate container. Cannot get any" +
+ " matching pipeline for Type:" + type +
+ ", Factor:" + replicationFactor + ", State:PipelineState.OPEN");
+ }
+ pipeline = pipelines.get((int) containerCount.get() % pipelines.size());
+ }
}
+
synchronized (pipeline) {
return allocateContainer(pipelineManager, owner, pipeline);
}
---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-commits-help@hadoop.apache.org