You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/07/13 19:19:05 UTC

[GitHub] [ozone] errose28 opened a new pull request, #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.

errose28 opened a new pull request, #3595:
URL: https://github.com/apache/ozone/pull/3595

   ## What changes were proposed in this pull request?
   
   Allow HDDS Layout features to specify:
   1. How many datanodes SCM should wait to finalize before declaring finalization complete and finalizing remaining datanodes asynchronously.
   2. Whether the layout feature requires all pipelines to be closed while it is finalizing.
   
   ## What is the link to the Apache JIRA
   
   HDDS-6885
   
   ## How was this patch tested?
   
   - Unit test for different finalization requirement combinations added.
   - Existing finalization tests should pass.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 commented on a diff in pull request #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.

Posted by GitBox <gi...@apache.org>.
errose28 commented on code in PR #3595:
URL: https://github.com/apache/ozone/pull/3595#discussion_r920518237


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java:
##########
@@ -22,32 +22,57 @@
 import java.util.Optional;
 
 import org.apache.hadoop.ozone.upgrade.LayoutFeature;
+import org.apache.hadoop.hdds.upgrade.HDDSFinalizationRequirements.PipelineRequirements;
 
 /**
  * List of HDDS Features.
  */
 public enum HDDSLayoutFeature implements LayoutFeature {
   //////////////////////////////  //////////////////////////////
-  INITIAL_VERSION(0, "Initial Layout Version"),
-  DATANODE_SCHEMA_V2(1, "Datanode RocksDB Schema Version 2 (with column " +
+
+  // TODO: After HDDS-6887 schema version changes will not require closing
+  //  pipelines.
+  INITIAL_VERSION(0,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Initial Layout Version"),
+  DATANODE_SCHEMA_V2(1,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)
+          .build(),
+      "Datanode RocksDB Schema Version 2 (with column " +
       "families)"),
-  SCM_HA(2, "Storage Container Manager HA"),
-  ERASURE_CODED_STORAGE_SUPPORT(3, "Ozone version with built in support for"
+  SCM_HA(2,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Storage Container Manager HA"),
+  ERASURE_CODED_STORAGE_SUPPORT(3,
+      new HDDSFinalizationRequirements.Builder()
+          // At least 5 datanodes are required for an EC pipeline.
+          .setMinFinalizedDatanodes(5)
+          .build(),
+      "Ozone version with built in support for"
       + " Erasure Coded block data storage."),
-  DATANODE_SCHEMA_V3(4, "Datanode RocksDB Schema Version 3 (one rocksdb " +
+  DATANODE_SCHEMA_V3(4,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)

Review Comment:
   That's right, pipelines would only be closed once. See the constructor in `HDDSLayoutVersionManager` and `HDDSFinalizationRequirements(Collection<HDDSFinalizationRequirements>)`. The requirements of all unfinalized layout features are aggregated to form one requirements object. This new object is what is acted upon during finalization.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 commented on a diff in pull request #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.

Posted by GitBox <gi...@apache.org>.
errose28 commented on code in PR #3595:
URL: https://github.com/apache/ozone/pull/3595#discussion_r920750712


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java:
##########
@@ -22,32 +22,57 @@
 import java.util.Optional;
 
 import org.apache.hadoop.ozone.upgrade.LayoutFeature;
+import org.apache.hadoop.hdds.upgrade.HDDSFinalizationRequirements.PipelineRequirements;
 
 /**
  * List of HDDS Features.
  */
 public enum HDDSLayoutFeature implements LayoutFeature {
   //////////////////////////////  //////////////////////////////
-  INITIAL_VERSION(0, "Initial Layout Version"),
-  DATANODE_SCHEMA_V2(1, "Datanode RocksDB Schema Version 2 (with column " +
+
+  // TODO: After HDDS-6887 schema version changes will not require closing
+  //  pipelines.
+  INITIAL_VERSION(0,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Initial Layout Version"),
+  DATANODE_SCHEMA_V2(1,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)
+          .build(),
+      "Datanode RocksDB Schema Version 2 (with column " +
       "families)"),
-  SCM_HA(2, "Storage Container Manager HA"),
-  ERASURE_CODED_STORAGE_SUPPORT(3, "Ozone version with built in support for"
+  SCM_HA(2,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Storage Container Manager HA"),
+  ERASURE_CODED_STORAGE_SUPPORT(3,
+      new HDDSFinalizationRequirements.Builder()
+          // At least 5 datanodes are required for an EC pipeline.
+          .setMinFinalizedDatanodes(5)
+          .build(),
+      "Ozone version with built in support for"
       + " Erasure Coded block data storage."),
-  DATANODE_SCHEMA_V3(4, "Datanode RocksDB Schema Version 3 (one rocksdb " +
+  DATANODE_SCHEMA_V3(4,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)

Review Comment:
   If no layout features that need finalization have explicitly specified that all pipelines must be closed, then pipelines will not be closed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 commented on pull request #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.

Posted by GitBox <gi...@apache.org>.
errose28 commented on PR #3595:
URL: https://github.com/apache/ozone/pull/3595#issuecomment-1188292097

   Thanks for reviewing this @kerneltime. Let's put this on hold for now while re re-evaluate our approach to not closing pipelines during finalization.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swagle commented on a diff in pull request #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.

Posted by GitBox <gi...@apache.org>.
swagle commented on code in PR #3595:
URL: https://github.com/apache/ozone/pull/3595#discussion_r920514946


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java:
##########
@@ -22,32 +22,57 @@
 import java.util.Optional;
 
 import org.apache.hadoop.ozone.upgrade.LayoutFeature;
+import org.apache.hadoop.hdds.upgrade.HDDSFinalizationRequirements.PipelineRequirements;
 
 /**
  * List of HDDS Features.
  */
 public enum HDDSLayoutFeature implements LayoutFeature {
   //////////////////////////////  //////////////////////////////
-  INITIAL_VERSION(0, "Initial Layout Version"),
-  DATANODE_SCHEMA_V2(1, "Datanode RocksDB Schema Version 2 (with column " +
+
+  // TODO: After HDDS-6887 schema version changes will not require closing
+  //  pipelines.
+  INITIAL_VERSION(0,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Initial Layout Version"),
+  DATANODE_SCHEMA_V2(1,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)
+          .build(),
+      "Datanode RocksDB Schema Version 2 (with column " +
       "families)"),
-  SCM_HA(2, "Storage Container Manager HA"),
-  ERASURE_CODED_STORAGE_SUPPORT(3, "Ozone version with built in support for"
+  SCM_HA(2,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Storage Container Manager HA"),
+  ERASURE_CODED_STORAGE_SUPPORT(3,
+      new HDDSFinalizationRequirements.Builder()
+          // At least 5 datanodes are required for an EC pipeline.
+          .setMinFinalizedDatanodes(5)
+          .build(),
+      "Ozone version with built in support for"
       + " Erasure Coded block data storage."),
-  DATANODE_SCHEMA_V3(4, "Datanode RocksDB Schema Version 3 (one rocksdb " +
+  DATANODE_SCHEMA_V3(4,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)

Review Comment:
   Quick check: This would not close pipelines twice right? Meaning two features with new layout version both saying close pipelines should only trigger this op once.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 commented on a diff in pull request #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.

Posted by GitBox <gi...@apache.org>.
errose28 commented on code in PR #3595:
URL: https://github.com/apache/ozone/pull/3595#discussion_r920749844


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSFinalizationRequirements.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.upgrade;
+
+import java.util.Collection;
+
+import static org.apache.hadoop.hdds.upgrade.HDDSFinalizationRequirements.PipelineRequirements.CLOSE_ALL_PIPELINES;
+
+/**
+ * Used by layout features in {@link HDDSLayoutFeature} to specify
+ * requirements that SCM must enforce before, during, or after they finalize.
+ */
+public class HDDSFinalizationRequirements {
+  /**
+   * What each layout feature requires for pipelines while it is
+   * finalizing.
+   */
+  public enum PipelineRequirements {
+    /**
+     * The layout feature requires all pipelines to be closed while finalizing.
+     */
+    CLOSE_ALL_PIPELINES,
+    /**
+     * The layout feature has no special requirements for pipeline handling
+     * while it is finalizing.
+     */
+    NONE
+  }
+
+  private final int minFinalizedDatanodes;
+  private final PipelineRequirements pipelineRequirements;
+
+  private HDDSFinalizationRequirements(Builder builder) {
+    minFinalizedDatanodes = builder.minFinalizedDatanodes;
+    pipelineRequirements = builder.pipelineRequirements;
+  }
+
+  /**
+   * Generates one requirements object by aggregating multiple requirements.
+   * The requirements aggregate will reflect the strictest requirements of
+   * any individual requirements provided.
+   */
+  public HDDSFinalizationRequirements(
+      Collection<HDDSFinalizationRequirements> requirements) {
+    int currentMinFinalizedDatanodes = 0;
+    PipelineRequirements currentPipelineRequirements =
+        PipelineRequirements.NONE;
+
+    for (HDDSFinalizationRequirements req: requirements) {
+      // The minimum number of datanodes we must wait to finalize is the
+      // largest of the minimums of all layout features.
+      currentMinFinalizedDatanodes = Math.max(currentMinFinalizedDatanodes,
+          req.minFinalizedDatanodes);
+
+      if (req.pipelineRequirements == CLOSE_ALL_PIPELINES) {
+        currentPipelineRequirements = CLOSE_ALL_PIPELINES;
+      }
+    }
+
+    minFinalizedDatanodes = currentMinFinalizedDatanodes;
+    pipelineRequirements = currentPipelineRequirements;
+  }
+
+  /**
+   * @return The minimum number of datanodes that SCM must wait to have
+   * finalized before declaring finalization has finished. The remaining
+   * datanodes will finalize asynchronously.
+   */
+  public int getMinFinalizedDatanodes() {
+    return minFinalizedDatanodes;
+  }
+
+  public PipelineRequirements getPipelineRequirements() {
+    return pipelineRequirements;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("Pipeline requirements: %s%nMinimum number of " +
+        "finalized datanodes: %s", pipelineRequirements, minFinalizedDatanodes);
+  }
+
+  /**
+   * Builds an {@link HDDSFinalizationRequirements} object, using default
+   * values for unspecified requirements.
+   */
+  public static final class Builder {
+    private int minFinalizedDatanodes;
+    private PipelineRequirements pipelineRequirements;
+
+    public Builder() {
+      // Default values.
+      this.minFinalizedDatanodes = 3;

Review Comment:
   Yes. In a future release, once EC is finalized if there was a layout feature that required closing all the pipelines we would need to wait for 6 nodes to finalize before an EC pipeline could be created.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] swagle commented on a diff in pull request #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.

Posted by GitBox <gi...@apache.org>.
swagle commented on code in PR #3595:
URL: https://github.com/apache/ozone/pull/3595#discussion_r920583928


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java:
##########
@@ -22,32 +22,57 @@
 import java.util.Optional;
 
 import org.apache.hadoop.ozone.upgrade.LayoutFeature;
+import org.apache.hadoop.hdds.upgrade.HDDSFinalizationRequirements.PipelineRequirements;
 
 /**
  * List of HDDS Features.
  */
 public enum HDDSLayoutFeature implements LayoutFeature {
   //////////////////////////////  //////////////////////////////
-  INITIAL_VERSION(0, "Initial Layout Version"),
-  DATANODE_SCHEMA_V2(1, "Datanode RocksDB Schema Version 2 (with column " +
+
+  // TODO: After HDDS-6887 schema version changes will not require closing
+  //  pipelines.
+  INITIAL_VERSION(0,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Initial Layout Version"),
+  DATANODE_SCHEMA_V2(1,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)
+          .build(),
+      "Datanode RocksDB Schema Version 2 (with column " +
       "families)"),
-  SCM_HA(2, "Storage Container Manager HA"),
-  ERASURE_CODED_STORAGE_SUPPORT(3, "Ozone version with built in support for"
+  SCM_HA(2,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Storage Container Manager HA"),
+  ERASURE_CODED_STORAGE_SUPPORT(3,
+      new HDDSFinalizationRequirements.Builder()
+          // At least 5 datanodes are required for an EC pipeline.
+          .setMinFinalizedDatanodes(5)
+          .build(),
+      "Ozone version with built in support for"
       + " Erasure Coded block data storage."),
-  DATANODE_SCHEMA_V3(4, "Datanode RocksDB Schema Version 3 (one rocksdb " +
+  DATANODE_SCHEMA_V3(4,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)

Review Comment:
   Sounds good thanks 👍



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kerneltime commented on a diff in pull request #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.

Posted by GitBox <gi...@apache.org>.
kerneltime commented on code in PR #3595:
URL: https://github.com/apache/ozone/pull/3595#discussion_r920610335


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSFinalizationRequirements.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.upgrade;
+
+import java.util.Collection;
+
+import static org.apache.hadoop.hdds.upgrade.HDDSFinalizationRequirements.PipelineRequirements.CLOSE_ALL_PIPELINES;
+
+/**
+ * Used by layout features in {@link HDDSLayoutFeature} to specify
+ * requirements that SCM must enforce before, during, or after they finalize.
+ */
+public class HDDSFinalizationRequirements {
+  /**
+   * What each layout feature requires for pipelines while it is
+   * finalizing.
+   */
+  public enum PipelineRequirements {
+    /**
+     * The layout feature requires all pipelines to be closed while finalizing.
+     */
+    CLOSE_ALL_PIPELINES,
+    /**
+     * The layout feature has no special requirements for pipeline handling
+     * while it is finalizing.
+     */
+    NONE
+  }
+
+  private final int minFinalizedDatanodes;
+  private final PipelineRequirements pipelineRequirements;
+
+  private HDDSFinalizationRequirements(Builder builder) {
+    minFinalizedDatanodes = builder.minFinalizedDatanodes;
+    pipelineRequirements = builder.pipelineRequirements;
+  }
+
+  /**
+   * Generates one requirements object by aggregating multiple requirements.
+   * The requirements aggregate will reflect the strictest requirements of
+   * any individual requirements provided.
+   */
+  public HDDSFinalizationRequirements(
+      Collection<HDDSFinalizationRequirements> requirements) {
+    int currentMinFinalizedDatanodes = 0;
+    PipelineRequirements currentPipelineRequirements =
+        PipelineRequirements.NONE;
+
+    for (HDDSFinalizationRequirements req: requirements) {
+      // The minimum number of datanodes we must wait to finalize is the
+      // largest of the minimums of all layout features.
+      currentMinFinalizedDatanodes = Math.max(currentMinFinalizedDatanodes,
+          req.minFinalizedDatanodes);
+
+      if (req.pipelineRequirements == CLOSE_ALL_PIPELINES) {
+        currentPipelineRequirements = CLOSE_ALL_PIPELINES;
+      }
+    }
+
+    minFinalizedDatanodes = currentMinFinalizedDatanodes;
+    pipelineRequirements = currentPipelineRequirements;
+  }
+
+  /**
+   * @return The minimum number of datanodes that SCM must wait to have
+   * finalized before declaring finalization has finished. The remaining
+   * datanodes will finalize asynchronously.
+   */
+  public int getMinFinalizedDatanodes() {
+    return minFinalizedDatanodes;
+  }
+
+  public PipelineRequirements getPipelineRequirements() {
+    return pipelineRequirements;
+  }
+
+  @Override
+  public String toString() {
+    return String.format("Pipeline requirements: %s%nMinimum number of " +
+        "finalized datanodes: %s", pipelineRequirements, minFinalizedDatanodes);
+  }
+
+  /**
+   * Builds an {@link HDDSFinalizationRequirements} object, using default
+   * values for unspecified requirements.
+   */
+  public static final class Builder {
+    private int minFinalizedDatanodes;
+    private PipelineRequirements pipelineRequirements;
+
+    public Builder() {
+      // Default values.
+      this.minFinalizedDatanodes = 3;

Review Comment:
   We would need to revisit this post enablement of EC



##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java:
##########
@@ -22,32 +22,57 @@
 import java.util.Optional;
 
 import org.apache.hadoop.ozone.upgrade.LayoutFeature;
+import org.apache.hadoop.hdds.upgrade.HDDSFinalizationRequirements.PipelineRequirements;
 
 /**
  * List of HDDS Features.
  */
 public enum HDDSLayoutFeature implements LayoutFeature {
   //////////////////////////////  //////////////////////////////
-  INITIAL_VERSION(0, "Initial Layout Version"),
-  DATANODE_SCHEMA_V2(1, "Datanode RocksDB Schema Version 2 (with column " +
+
+  // TODO: After HDDS-6887 schema version changes will not require closing
+  //  pipelines.
+  INITIAL_VERSION(0,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Initial Layout Version"),
+  DATANODE_SCHEMA_V2(1,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)
+          .build(),
+      "Datanode RocksDB Schema Version 2 (with column " +
       "families)"),
-  SCM_HA(2, "Storage Container Manager HA"),
-  ERASURE_CODED_STORAGE_SUPPORT(3, "Ozone version with built in support for"
+  SCM_HA(2,
+      new HDDSFinalizationRequirements.Builder().build(),
+      "Storage Container Manager HA"),
+  ERASURE_CODED_STORAGE_SUPPORT(3,
+      new HDDSFinalizationRequirements.Builder()
+          // At least 5 datanodes are required for an EC pipeline.
+          .setMinFinalizedDatanodes(5)
+          .build(),
+      "Ozone version with built in support for"
       + " Erasure Coded block data storage."),
-  DATANODE_SCHEMA_V3(4, "Datanode RocksDB Schema Version 3 (one rocksdb " +
+  DATANODE_SCHEMA_V3(4,
+      new HDDSFinalizationRequirements.Builder()
+          .setPipelineRequirements(PipelineRequirements.CLOSE_ALL_PIPELINES)

Review Comment:
   Will all pipelines be closed if this feature is defaulted to off?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] errose28 closed pull request #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.

Posted by "errose28 (via GitHub)" <gi...@apache.org>.
errose28 closed pull request #3595: HDDS-6885. Allow HDDS layout features to specify their finalization requirements.
URL: https://github.com/apache/ozone/pull/3595


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org