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 2020/12/02 22:58:32 UTC

[GitHub] [ozone] prashantpogde opened a new pull request #1651: Hdds 4178

prashantpogde opened a new pull request #1651:
URL: https://github.com/apache/ozone/pull/1651


   ## What changes were proposed in this pull request?
   
   SCM Finalize command implementation
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-4178
   
   ## How was this patch tested?
   
   UT. I will fix CI failures.
   


----------------------------------------------------------------
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.

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] avijayanhwx commented on a change in pull request #1651: HDDS-4178. SCM Finalize command implementation

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r536386166



##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/upgrade/FinalizeScmUpgradeSubcommand.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scm.cli.upgrade;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.ozone.upgrade.UpgradeException;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
+
+import picocli.CommandLine;
+
+/**
+ * Handler of Finalize SCM command.
+ */
+@CommandLine.Command(
+    name = "FinalizeScm",
+    description = "Finalize SCM Upgrade",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+public class FinalizeScmUpgradeSubcommand extends ScmSubcommand {

Review comment:
       With this patch I see the CLI different for OM and SCM.
   
   For SCM, it is something like **_ozone admin upgrade FinalizeSCM_**
   For OM, it is **_ozone admin om finalizeupgrade_**
   
   Can we make sure it is common across OM and SCM? I believe the following are OK -> 
   _ozone admin om/scm finalizeupgrade
   ozone admin upgrade finalize-om-upgrade/finalize-scm-upgrade_
   
   In Ozone, the convention is to have CLI operations should be lower case with "-" between words as needed.

##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ScmSubcommand.java
##########
@@ -17,21 +17,25 @@
  */
 package org.apache.hadoop.hdds.scm.cli;
 
+import org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeBaseCommand;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
 import picocli.CommandLine;
 
 import java.io.IOException;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 
 /**
  * Base class for admin commands that connect via SCM client.
  */
-public abstract class ScmSubcommand implements Callable<Void> {
+public abstract class ScmSubcommand extends FinalizeUpgradeBaseCommand

Review comment:
       Why do we need ScmSubCommand to extend FinalizeUpgradeBaseCommand? Shouldn't the inheritance be the other way? 

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -211,6 +214,7 @@
   private PipelineChoosePolicy pipelineChoosePolicy;
 
   private HDDSLayoutVersionManager scmLayoutVersionManager;
+  private UpgradeFinalizer upgradeFinalizer;

Review comment:
       Nit. Can be UpgradeFinalizer\<StorageContainerManager\>. 




----------------------------------------------------------------
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.

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] fapifta commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r540598745



##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/upgrade/FinalizeScmUpgradeSubcommand.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scm.cli.upgrade;
+
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitCancellationMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitExitMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitFinishedMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitGeneralErrorMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.handleInvalidRequestAfterInitiatingFinalization;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isDone;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isFinalized;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isInprogress;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isStarting;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.ozone.upgrade.UpgradeException;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
+
+import picocli.CommandLine;
+
+/**
+ * Handler of Finalize SCM command.
+ */
+@CommandLine.Command(
+    name = "finalizeupgrade",
+    description = "Finalize SCM Upgrade",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+public class FinalizeScmUpgradeSubcommand extends ScmSubcommand {
+  @CommandLine.Option(
+      names = {"--takeover"},
+      description = "Forces takeover of monitoring from another client, if "
+          + "finalization has already been started and did not finish yet."
+  )
+  private boolean force;
+
+  @Override
+  public void execute(ScmClient scmClient)
+      throws IOException, ExecutionException {
+    String upgradeClientID = "Upgrade-Client-" + UUID.randomUUID().toString();
+    try {
+      StatusAndMessages finalizationResponse =
+          scmClient.finalizeScmUpgrade(upgradeClientID);
+      if (isFinalized(finalizationResponse.status())){
+        System.out.println("Upgrade has already been finalized.");
+        emitExitMsg();
+        return;
+      } else if (!isStarting(finalizationResponse.status())){
+        System.err.println("Invalid response from Storage Container Manager.");
+        System.err.println(
+            "Current finalization status is: " + finalizationResponse.status()
+        );
+        throw new IOException("Exiting...");
+      }
+    } catch (UpgradeException e) {
+      handleInvalidRequestAfterInitiatingFinalization(force, e);
+    }
+    monitorAndWaitFinalization(scmClient, upgradeClientID);
+    return;
+  }
+
+  private void monitorAndWaitFinalization(ScmClient client,
+                                          String upgradeClientID) throws
+      ExecutionException {
+    ExecutorService exec = Executors.newSingleThreadExecutor();
+    Future<?> monitor =
+        exec.submit(new UpgradeMonitor(client, upgradeClientID, force));
+    try {
+      monitor.get();
+      emitFinishedMsg("Storage Container Manager");
+    } catch (CancellationException |InterruptedException e) {
+      emitCancellationMsg("Storage Container Manager");
+    } catch (ExecutionException e) {
+      emitGeneralErrorMsg();
+      throw e;

Review comment:
       I think, here we should just throw the cause of the ExecutionException wrapped into an IOException here.
   I believe it is a better approach, as with that other subcommands are not forced to declare to throw two types of exceptions.
   
   However as an alternative I am absolutely fine with declaring execute to throw an Exception, and do not mark specific Exceptions to be thrown, as the problem I see is the number of exceptions that method needs to declare and seems to declare implicitly for all SCM subcommands.




----------------------------------------------------------------
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.

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] prashantpogde commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r539691257



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -211,6 +214,7 @@
   private PipelineChoosePolicy pipelineChoosePolicy;
 
   private HDDSLayoutVersionManager scmLayoutVersionManager;
+  private UpgradeFinalizer upgradeFinalizer;

Review comment:
       yup. Made the changes as suggested.




----------------------------------------------------------------
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.

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] prashantpogde commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r541163388



##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/upgrade/FinalizeScmUpgradeSubcommand.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scm.cli.upgrade;
+
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitCancellationMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitExitMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitFinishedMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitGeneralErrorMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.handleInvalidRequestAfterInitiatingFinalization;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isDone;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isFinalized;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isInprogress;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isStarting;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.ozone.upgrade.UpgradeException;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
+
+import picocli.CommandLine;
+
+/**
+ * Handler of Finalize SCM command.
+ */
+@CommandLine.Command(
+    name = "finalizeupgrade",
+    description = "Finalize SCM Upgrade",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+public class FinalizeScmUpgradeSubcommand extends ScmSubcommand {
+  @CommandLine.Option(
+      names = {"--takeover"},
+      description = "Forces takeover of monitoring from another client, if "
+          + "finalization has already been started and did not finish yet."
+  )
+  private boolean force;
+
+  @Override
+  public void execute(ScmClient scmClient) throws IOException {
+    String upgradeClientID = "Upgrade-Client-" + UUID.randomUUID().toString();
+    try {
+      StatusAndMessages finalizationResponse =
+          scmClient.finalizeScmUpgrade(upgradeClientID);
+      if (isFinalized(finalizationResponse.status())){
+        System.out.println("Upgrade has already been finalized.");
+        emitExitMsg();
+        return;
+      } else if (!isStarting(finalizationResponse.status())){
+        System.err.println("Invalid response from Storage Container Manager.");
+        System.err.println(
+            "Current finalization status is: " + finalizationResponse.status()
+        );
+        throw new IOException("Exiting...");
+      }
+    } catch (UpgradeException e) {
+      handleInvalidRequestAfterInitiatingFinalization(force, e);
+    }
+    monitorAndWaitFinalization(scmClient, upgradeClientID);
+    return;
+  }
+
+  private void monitorAndWaitFinalization(ScmClient client,
+                                          String upgradeClientID)
+      throws IOException {
+    ExecutorService exec = Executors.newSingleThreadExecutor();
+    Future<?> monitor =
+        exec.submit(new UpgradeMonitor(client, upgradeClientID, force));
+    try {
+      monitor.get();
+      emitFinishedMsg("Storage Container Manager");
+    } catch (CancellationException |InterruptedException e) {
+      emitCancellationMsg("Storage Container Manager");
+    } catch (ExecutionException e) {
+      emitGeneralErrorMsg();
+      throw new IOException(e.getMessage());

Review comment:
       Done.




----------------------------------------------------------------
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.

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] avijayanhwx commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r536386166



##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/upgrade/FinalizeScmUpgradeSubcommand.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scm.cli.upgrade;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.ozone.upgrade.UpgradeException;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
+
+import picocli.CommandLine;
+
+/**
+ * Handler of Finalize SCM command.
+ */
+@CommandLine.Command(
+    name = "FinalizeScm",
+    description = "Finalize SCM Upgrade",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+public class FinalizeScmUpgradeSubcommand extends ScmSubcommand {

Review comment:
       With this patch I see the CLI different for OM and SCM.
   
   For SCM, it is something like **_ozone admin upgrade FinalizeSCM_**
   For OM, it is **_ozone admin om finalizeupgrade_**
   
   Can we make sure it is common across OM and SCM? I believe the following are OK -> 
   _ozone admin om/scm finalizeupgrade
   ozone admin upgrade finalize-om-upgrade/finalize-scm-upgrade_
   
   In Ozone, the convention is to have CLI operations use lower case with "-" between words as needed.




----------------------------------------------------------------
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.

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] avijayanhwx merged pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
avijayanhwx merged pull request #1651:
URL: https://github.com/apache/ozone/pull/1651


   


----------------------------------------------------------------
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.

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] prashantpogde commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r540748686



##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/upgrade/FinalizeScmUpgradeSubcommand.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scm.cli.upgrade;
+
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitCancellationMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitExitMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitFinishedMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitGeneralErrorMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.handleInvalidRequestAfterInitiatingFinalization;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isDone;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isFinalized;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isInprogress;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isStarting;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.ozone.upgrade.UpgradeException;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
+
+import picocli.CommandLine;
+
+/**
+ * Handler of Finalize SCM command.
+ */
+@CommandLine.Command(
+    name = "finalizeupgrade",
+    description = "Finalize SCM Upgrade",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+public class FinalizeScmUpgradeSubcommand extends ScmSubcommand {
+  @CommandLine.Option(
+      names = {"--takeover"},
+      description = "Forces takeover of monitoring from another client, if "
+          + "finalization has already been started and did not finish yet."
+  )
+  private boolean force;
+
+  @Override
+  public void execute(ScmClient scmClient)
+      throws IOException, ExecutionException {
+    String upgradeClientID = "Upgrade-Client-" + UUID.randomUUID().toString();
+    try {
+      StatusAndMessages finalizationResponse =
+          scmClient.finalizeScmUpgrade(upgradeClientID);
+      if (isFinalized(finalizationResponse.status())){
+        System.out.println("Upgrade has already been finalized.");
+        emitExitMsg();
+        return;
+      } else if (!isStarting(finalizationResponse.status())){
+        System.err.println("Invalid response from Storage Container Manager.");
+        System.err.println(
+            "Current finalization status is: " + finalizationResponse.status()
+        );
+        throw new IOException("Exiting...");
+      }
+    } catch (UpgradeException e) {
+      handleInvalidRequestAfterInitiatingFinalization(force, e);
+    }
+    monitorAndWaitFinalization(scmClient, upgradeClientID);
+    return;
+  }
+
+  private void monitorAndWaitFinalization(ScmClient client,
+                                          String upgradeClientID) throws
+      ExecutionException {
+    ExecutorService exec = Executors.newSingleThreadExecutor();
+    Future<?> monitor =
+        exec.submit(new UpgradeMonitor(client, upgradeClientID, force));
+    try {
+      monitor.get();
+      emitFinishedMsg("Storage Container Manager");
+    } catch (CancellationException |InterruptedException e) {
+      emitCancellationMsg("Storage Container Manager");
+    } catch (ExecutionException e) {
+      emitGeneralErrorMsg();
+      throw e;

Review comment:
       yup, I will make the change.




----------------------------------------------------------------
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.

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] prashantpogde commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r539693427



##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/upgrade/FinalizeScmUpgradeSubcommand.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scm.cli.upgrade;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.ozone.upgrade.UpgradeException;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
+
+import picocli.CommandLine;
+
+/**
+ * Handler of Finalize SCM command.
+ */
+@CommandLine.Command(
+    name = "FinalizeScm",
+    description = "Finalize SCM Upgrade",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+public class FinalizeScmUpgradeSubcommand extends ScmSubcommand {

Review comment:
       yes! I changed  it to
   
   ozone admin om/scm finalizeupgrade
   




----------------------------------------------------------------
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.

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] avijayanhwx commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r539731996



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/BasicUpgradeFinalizer.java
##########
@@ -218,14 +218,15 @@ private void persistStorage(Storage config) throws IOException {
 
   protected void emitNOOPMsg(String feature) {
     String msg = "No finalization work defined for feature: " + feature + ".";
-    String msg2 = "Skipping.";
 
     logAndEmit(msg);
-    logAndEmit(msg2);
   }
 
   protected void emitStartingMsg() {
-    String msg = "Finalization started.";
+    String msg = "Finalization started.\n";
+    msg += "Existing pipelines and containers will be closed during Upgrade.\n";

Review comment:
       If we add here, then OM finalization will also print out the message. Can we print out the message after the actual work is done (freeze pipeline creation, pipelines are closed etc)? 




----------------------------------------------------------------
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.

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] avijayanhwx commented on pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#issuecomment-744761556


   Merging this since CI failure is unrelated.


----------------------------------------------------------------
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.

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] fapifta commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r540598745



##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/upgrade/FinalizeScmUpgradeSubcommand.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scm.cli.upgrade;
+
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitCancellationMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitExitMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitFinishedMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitGeneralErrorMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.handleInvalidRequestAfterInitiatingFinalization;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isDone;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isFinalized;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isInprogress;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isStarting;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.ozone.upgrade.UpgradeException;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
+
+import picocli.CommandLine;
+
+/**
+ * Handler of Finalize SCM command.
+ */
+@CommandLine.Command(
+    name = "finalizeupgrade",
+    description = "Finalize SCM Upgrade",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+public class FinalizeScmUpgradeSubcommand extends ScmSubcommand {
+  @CommandLine.Option(
+      names = {"--takeover"},
+      description = "Forces takeover of monitoring from another client, if "
+          + "finalization has already been started and did not finish yet."
+  )
+  private boolean force;
+
+  @Override
+  public void execute(ScmClient scmClient)
+      throws IOException, ExecutionException {
+    String upgradeClientID = "Upgrade-Client-" + UUID.randomUUID().toString();
+    try {
+      StatusAndMessages finalizationResponse =
+          scmClient.finalizeScmUpgrade(upgradeClientID);
+      if (isFinalized(finalizationResponse.status())){
+        System.out.println("Upgrade has already been finalized.");
+        emitExitMsg();
+        return;
+      } else if (!isStarting(finalizationResponse.status())){
+        System.err.println("Invalid response from Storage Container Manager.");
+        System.err.println(
+            "Current finalization status is: " + finalizationResponse.status()
+        );
+        throw new IOException("Exiting...");
+      }
+    } catch (UpgradeException e) {
+      handleInvalidRequestAfterInitiatingFinalization(force, e);
+    }
+    monitorAndWaitFinalization(scmClient, upgradeClientID);
+    return;
+  }
+
+  private void monitorAndWaitFinalization(ScmClient client,
+                                          String upgradeClientID) throws
+      ExecutionException {
+    ExecutorService exec = Executors.newSingleThreadExecutor();
+    Future<?> monitor =
+        exec.submit(new UpgradeMonitor(client, upgradeClientID, force));
+    try {
+      monitor.get();
+      emitFinishedMsg("Storage Container Manager");
+    } catch (CancellationException |InterruptedException e) {
+      emitCancellationMsg("Storage Container Manager");
+    } catch (ExecutionException e) {
+      emitGeneralErrorMsg();
+      throw e;

Review comment:
       I think, here we should just throw the cause of the ExecutionException wrapped into an IOException here.
   I believe it is a better approach, as with that other subcommands are not forced to declare to throw two types of exceptions.
   
   However as an alternative I am absolutely fine with declaring execute to throw an Exception, and do not mark specific Exceptions to be thrown.




----------------------------------------------------------------
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.

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] prashantpogde commented on pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#issuecomment-742853039


   acceptance failure is unrelated.


----------------------------------------------------------------
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.

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] prashantpogde commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r539775435



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/BasicUpgradeFinalizer.java
##########
@@ -218,14 +218,15 @@ private void persistStorage(Storage config) throws IOException {
 
   protected void emitNOOPMsg(String feature) {
     String msg = "No finalization work defined for feature: " + feature + ".";
-    String msg2 = "Skipping.";
 
     logAndEmit(msg);
-    logAndEmit(msg2);
   }
 
   protected void emitStartingMsg() {
-    String msg = "Finalization started.";
+    String msg = "Finalization started.\n";
+    msg += "Existing pipelines and containers will be closed during Upgrade.\n";

Review comment:
       yup, I thought I was making change in ScmFinalizer. Fixed now.




----------------------------------------------------------------
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.

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] fapifta commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
fapifta commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r540791173



##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/upgrade/FinalizeScmUpgradeSubcommand.java
##########
@@ -0,0 +1,152 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.scm.cli.upgrade;
+
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitCancellationMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitExitMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitFinishedMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.emitGeneralErrorMsg;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.handleInvalidRequestAfterInitiatingFinalization;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isDone;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isFinalized;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isInprogress;
+import static org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeCommandUtil.isStarting;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.apache.hadoop.ozone.upgrade.UpgradeException;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.StatusAndMessages;
+
+import picocli.CommandLine;
+
+/**
+ * Handler of Finalize SCM command.
+ */
+@CommandLine.Command(
+    name = "finalizeupgrade",
+    description = "Finalize SCM Upgrade",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+public class FinalizeScmUpgradeSubcommand extends ScmSubcommand {
+  @CommandLine.Option(
+      names = {"--takeover"},
+      description = "Forces takeover of monitoring from another client, if "
+          + "finalization has already been started and did not finish yet."
+  )
+  private boolean force;
+
+  @Override
+  public void execute(ScmClient scmClient) throws IOException {
+    String upgradeClientID = "Upgrade-Client-" + UUID.randomUUID().toString();
+    try {
+      StatusAndMessages finalizationResponse =
+          scmClient.finalizeScmUpgrade(upgradeClientID);
+      if (isFinalized(finalizationResponse.status())){
+        System.out.println("Upgrade has already been finalized.");
+        emitExitMsg();
+        return;
+      } else if (!isStarting(finalizationResponse.status())){
+        System.err.println("Invalid response from Storage Container Manager.");
+        System.err.println(
+            "Current finalization status is: " + finalizationResponse.status()
+        );
+        throw new IOException("Exiting...");
+      }
+    } catch (UpgradeException e) {
+      handleInvalidRequestAfterInitiatingFinalization(force, e);
+    }
+    monitorAndWaitFinalization(scmClient, upgradeClientID);
+    return;
+  }
+
+  private void monitorAndWaitFinalization(ScmClient client,
+                                          String upgradeClientID)
+      throws IOException {
+    ExecutorService exec = Executors.newSingleThreadExecutor();
+    Future<?> monitor =
+        exec.submit(new UpgradeMonitor(client, upgradeClientID, force));
+    try {
+      monitor.get();
+      emitFinishedMsg("Storage Container Manager");
+    } catch (CancellationException |InterruptedException e) {
+      emitCancellationMsg("Storage Container Manager");
+    } catch (ExecutionException e) {
+      emitGeneralErrorMsg();
+      throw new IOException(e.getMessage());

Review comment:
       Thank you for making this change, one more thing still regarding this part, I think it is more beneficial to do not swallow the stack trace of this exception and having it reported in the output, so can we provide e.getCause() instead of e.getMessage() here to the wrapping IOException?




----------------------------------------------------------------
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.

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] prashantpogde commented on a change in pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on a change in pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#discussion_r539691637



##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ScmSubcommand.java
##########
@@ -17,21 +17,25 @@
  */
 package org.apache.hadoop.hdds.scm.cli;
 
+import org.apache.hadoop.hdds.scm.cli.upgrade.FinalizeUpgradeBaseCommand;
 import org.apache.hadoop.hdds.scm.client.ScmClient;
 import picocli.CommandLine;
 
 import java.io.IOException;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 
 /**
  * Base class for admin commands that connect via SCM client.
  */
-public abstract class ScmSubcommand implements Callable<Void> {
+public abstract class ScmSubcommand extends FinalizeUpgradeBaseCommand

Review comment:
       I changed the structure as discussed over call so this is not applicable anymore.




----------------------------------------------------------------
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.

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] prashantpogde commented on pull request #1651: HDDS-4178. SCM Finalize client command implementation

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on pull request #1651:
URL: https://github.com/apache/ozone/pull/1651#issuecomment-743036803


   thanks @fapifta @avijayanhwx for the review.


----------------------------------------------------------------
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.

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