You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "tanvipenumudy (via GitHub)" <gi...@apache.org> on 2023/05/19 13:09:46 UTC

[GitHub] [ozone] tanvipenumudy opened a new pull request, #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

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

   ## What changes were proposed in this pull request?
   
   Introduce an Ozone admin SCM CLI command to force generate new secret keys (rotate).
   
   Format: `ozone admin scm rotate --scm hostPort`
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-8659
   
   ## How was this patch tested?
   
   Added integration test cases.
   


-- 
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] tanvipenumudy commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1259153834


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeyManager.java:
##########
@@ -126,6 +127,8 @@ public synchronized boolean checkAndRotate() throws SCMException {
       state.updateKeys(updatedKeys);
       return true;
     }
+    LOG.info("The latest key was created at: " + currentKey.getCreationTime() +

Review Comment:
   Understood, will remove this log message, 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] duongkame commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "duongkame (via GitHub)" <gi...@apache.org>.
duongkame commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1237535524


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/SecretKeyProtocolServerSideTranslatorPB.java:
##########
@@ -102,6 +104,21 @@ public SCMSecretKeyResponse processRequest(SCMSecretKeyRequest request)
             .setSecretKeysListResponseProto(getAllSecretKeys())
             .build();
 
+      case GetCheckAndRotate:
+        try {
+          return scmSecurityResponse
+              .setCheckAndRotateResponseProto(
+                  checkAndRotate(request.getCheckAndRotateRequest().getForce()))
+              .build();
+        } catch (TimeoutException e) {

Review Comment:
   This exception handling can happen in `SCMSecurityProtocolServer.checkAndRotate`, so that we can avoid TimeoutException in the interface level.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/RotateKeySubCommand.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.ozone.admin.scm;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+
+/**
+ * Handler of ozone admin scm rotate command.
+ */
+@CommandLine.Command(
+    name = "rotate",
+    description = "CLI command to force generate new keys (rotate)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class RotateKeySubCommand extends ScmSubcommand {
+
+  @CommandLine.Option(names = "--force",
+      description = "Force generate new keys")
+      private boolean force = false;
+
+  @CommandLine.ParentCommand
+  private ScmAdmin parent;
+
+  @Override
+  protected void execute(ScmClient scmClient) throws IOException {
+    try (ScmClient client = new ContainerOperationClient(
+        parent.getParent().getOzoneConf())) {
+      boolean status = false;
+      try {
+        status = client.rotateSecretKeys(force);
+      } catch (TimeoutException e) {
+        System.err.println(
+            "Secret key rotation failed due to a timeout: " + e.getMessage());
+        return;
+      } catch (IOException e) {
+        System.err.println("Secret key rotation failed: " + e.getMessage());
+        return;
+      }
+      if (status) {
+        System.out.println("Secret key rotation is complete. A new key has " +
+            "been generated. Rotate Status: " + status);

Review Comment:
   ```suggestion
           System.out.println("Secret key rotation is complete. A new key has " +
               "been generated.");
   ```



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/protocol/SecretKeyProtocolScm.java:
##########
@@ -28,4 +31,14 @@
     clientPrincipal = HDDS_SCM_KERBEROS_PRINCIPAL_KEY
 )
 public interface SecretKeyProtocolScm extends SecretKeyProtocol {
+
+  /**
+   * Force generates new secret keys (rotate).
+   *
+   * @param force boolean flag that forcefully rotates the key on demand
+   * @return key rotation status
+   * @throws TimeoutException
+   * @throws IOException
+   */
+  boolean checkAndRotate(boolean force) throws TimeoutException, IOException;

Review Comment:
   As below comments, `TimeoutException` should not be exposed to client side.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java:
##########
@@ -232,6 +233,13 @@ private void validateSecretKeyStatus() throws SCMSecretKeyException {
     }
   }
 
+  @Override
+  public boolean checkAndRotate(boolean force)
+      throws TimeoutException, SCMSecretKeyException {
+    validateSecretKeyStatus();
+    return secretKeyManager.checkAndRotate(force);

Review Comment:
   ```suggestion
       try {
         return secretKeyManager.checkAndRotate(force);
       } catch (TimeoutException ex) {
         LOGGER.error("Timeout rotating secret keys", ex);
         throw new SCMSecretKeyException(ex.getMessage(), SCMSecretKeyException.ErrorCode.INTERNAL_ERROR);
       }
   ```



-- 
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] duongkame commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "duongkame (via GitHub)" <gi...@apache.org>.
duongkame commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1199481362


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java:
##########
@@ -232,6 +234,16 @@ private void validateSecretKeyStatus() throws SCMSecretKeyException {
     }
   }
 
+  @Override
+  public boolean checkAndRotate(boolean force) throws TimeoutException {
+    try {
+      validateSecretKeyStatus();
+    } catch (SCMSecretKeyException e) {
+      e.printStackTrace();

Review Comment:
   This error should be thrown to client-side, like other APIs. 



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/SecretKeyProtocolServerSideTranslatorPB.java:
##########
@@ -102,6 +108,16 @@ public SCMSecretKeyResponse processRequest(SCMSecretKeyRequest request)
             .setSecretKeysListResponseProto(getAllSecretKeys())
             .build();
 
+      case GetCheckAndRotate:
+        try {
+          return scmSecurityResponse
+              .setCheckAndRotateResponseProto(
+                  checkAndRotate(request.getCheckAndRotateRequest().getForce()))
+              .build();
+        } catch (TimeoutException e) {
+          e.printStackTrace();

Review Comment:
   +1 error should be handled by the client (CLI)



-- 
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] ashishkumar50 commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "ashishkumar50 (via GitHub)" <gi...@apache.org>.
ashishkumar50 commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1203769732


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokens.java:
##########
@@ -377,6 +386,23 @@ private static void setSecureConfig() throws IOException {
         ozoneKeytab.getAbsolutePath());
   }
 
+  @Test
+  public void testRotateKeySCMAdminCommand()
+      throws InterruptedException, TimeoutException, IOException {
+    GenericTestUtils.waitFor(() -> cluster.getScmLeader() != null, 100, 1000);
+    InetSocketAddress address =
+        cluster.getScmLeader().getClientRpcAddress();
+    String hostPort = address.getHostName() + ":" + address.getPort();
+    String[] args = {"scm", "rotate", "--scm", hostPort};
+
+    String oldKey =
+        scmClient.getSecretKeyClient().getCurrentSecretKey().toString();

Review Comment:
   May be we can add a small test here, run twice "getCurrentSecretKey" and assert whether both are same before rotating.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/RotateKeySubCommand.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ozone.admin.scm;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+
+/**
+ * Handler of ozone admin scm rotate command.
+ */
+@CommandLine.Command(
+    name = "rotate",
+    description = "CLI command to force generate new keys (rotate)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class RotateKeySubCommand extends ScmSubcommand {
+
+  @CommandLine.ParentCommand
+  private ScmAdmin parent;
+
+  @Override
+  protected void execute(ScmClient scmClient) throws IOException {
+    try (ScmClient client = new ContainerOperationClient(
+        parent.getParent().getOzoneConf())) {
+      boolean status = false;
+      try {
+        status = client.checkAndRotate(true);
+      } catch (TimeoutException e) {
+        e.printStackTrace();
+      }
+      System.out.println(
+          "Secret key rotation is complete, a new key has been generated. " +
+              "Rotate Status: " + status);

Review Comment:
   When there is TimeoutException, This message will not be correct to show. Instead we can print different message in catch and return from there.



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/protocol/SecretKeyProtocolScm.java:
##########
@@ -28,4 +31,13 @@
     clientPrincipal = HDDS_SCM_KERBEROS_PRINCIPAL_KEY
 )
 public interface SecretKeyProtocolScm extends SecretKeyProtocol {
+
+  /**
+   * Force generates new secret keys (rotate).
+   *
+   * @param force boolean flag that forcefully rotates the key on demand
+   * @return
+   * @throws TimeoutException

Review Comment:
   Please Correct javadoc for @return and @throws.



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeyManager.java:
##########
@@ -109,12 +109,13 @@ public boolean isInitialized() {
    *
    * @return true if rotation actually happens, false if it doesn't.
    */
-  public synchronized boolean checkAndRotate() throws TimeoutException {
+  public synchronized boolean checkAndRotate(boolean force)
+      throws TimeoutException {
     // Initialize the state if it's not initialized already.
     checkAndInitialize();
 
     ManagedSecretKey currentKey = state.getCurrentKey();
-    if (shouldRotate(currentKey)) {
+    if (shouldRotate(currentKey) || force) {

Review Comment:
   nit: we can reverse condition "if (force || shouldRotate(currentKey))", to avoid method call if force is true.



##########
hadoop-hdds/interface-server/src/main/proto/ScmSecretKeyProtocol.proto:
##########
@@ -98,6 +103,11 @@ message SCMGetSecretKeyRequest {
     required UUID secretKeyId = 1;
 }
 
+message SCMGetCheckAndRotateRequest {
+    optional bool force = 1;

Review Comment:
   We can change to optional bool force = 1 [default = false];



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/RotateKeySubCommand.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ozone.admin.scm;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+
+/**
+ * Handler of ozone admin scm rotate command.
+ */
+@CommandLine.Command(
+    name = "rotate",
+    description = "CLI command to force generate new keys (rotate)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class RotateKeySubCommand extends ScmSubcommand {
+
+  @CommandLine.ParentCommand
+  private ScmAdmin parent;
+
+  @Override
+  protected void execute(ScmClient scmClient) throws IOException {
+    try (ScmClient client = new ContainerOperationClient(
+        parent.getParent().getOzoneConf())) {
+      boolean status = false;
+      try {
+        status = client.checkAndRotate(true);
+      } catch (TimeoutException e) {

Review Comment:
   checkAndRotate can even throw IOException, we can handle and show proper message too.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/RotateKeySubCommand.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ozone.admin.scm;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+
+/**
+ * Handler of ozone admin scm rotate command.
+ */
+@CommandLine.Command(
+    name = "rotate",
+    description = "CLI command to force generate new keys (rotate)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class RotateKeySubCommand extends ScmSubcommand {
+
+  @CommandLine.ParentCommand
+  private ScmAdmin parent;
+
+  @Override
+  protected void execute(ScmClient scmClient) throws IOException {
+    try (ScmClient client = new ContainerOperationClient(
+        parent.getParent().getOzoneConf())) {
+      boolean status = false;
+      try {
+        status = client.checkAndRotate(true);
+      } catch (TimeoutException e) {
+        e.printStackTrace();
+      }
+      System.out.println(
+          "Secret key rotation is complete, a new key has been generated. " +
+              "Rotate Status: " + status);
+    }
+  }
+}

Review Comment:
   Add a new line at the end of file.



-- 
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] tanvipenumudy commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1259137599


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeyManager.java:
##########
@@ -109,12 +109,13 @@ public boolean isInitialized() {
    *
    * @return true if rotation actually happens, false if it doesn't.
    */
-  public synchronized boolean checkAndRotate() throws SCMException {
+  public synchronized boolean checkAndRotate(boolean force)
+      throws SCMException {
     // Initialize the state if it's not initialized already.
     checkAndInitialize();
 
     ManagedSecretKey currentKey = state.getCurrentKey();
-    if (shouldRotate(currentKey)) {
+    if (force || shouldRotate(currentKey)) {

Review Comment:
   Thanks @duongkame, I have added the log message for the same.



-- 
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] duongkame commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "duongkame (via GitHub)" <gi...@apache.org>.
duongkame commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1199480521


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/protocolPB/SecretKeyProtocolClientSideTranslatorPB.java:
##########
@@ -129,6 +131,23 @@ public ManagedSecretKey getCurrentSecretKey() throws IOException {
     return ManagedSecretKey.fromProtobuf(secretKeyProto);
   }
 
+  @Override
+  public boolean checkAndRotate(boolean force) throws TimeoutException {
+    boolean checkAndRotateStatus = false;
+    try {
+      SCMSecretKeyProtocolProtos.SCMGetCheckAndRotateRequest request =
+          SCMSecretKeyProtocolProtos.SCMGetCheckAndRotateRequest.newBuilder()
+              .setForce(force)
+              .build();
+      checkAndRotateStatus = submitRequest(Type.GetCheckAndRotate, builder ->
+          builder.setCheckAndRotateRequest(request))
+          .getCheckAndRotateResponseProto().getStatus();
+    } catch (IOException e) {
+      e.printStackTrace();

Review Comment:
   should be thrown and handled outside. 



-- 
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] duongkame commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "duongkame (via GitHub)" <gi...@apache.org>.
duongkame commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1258902441


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeyManager.java:
##########
@@ -126,6 +127,8 @@ public synchronized boolean checkAndRotate() throws SCMException {
       state.updateKeys(updatedKeys);
       return true;
     }
+    LOG.info("The latest key was created at: " + currentKey.getCreationTime() +

Review Comment:
   We may not need this log as INFO, as it'll blood the log files with not-very-useful messages. Note that this function is called by a periodic schedule.



-- 
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] tanvipenumudy commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1204063813


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/RotateKeySubCommand.java:
##########
@@ -0,0 +1,40 @@
+package org.apache.hadoop.ozone.admin.scm;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+
+/**
+ * Handler of ozone admin scm rotate command.
+ */
+@CommandLine.Command(
+    name = "rotate",
+    description = "CLI command to force generate new keys (rotate)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class RotateKeySubCommand extends ScmSubcommand {
+
+  @CommandLine.ParentCommand
+  private ScmAdmin parent;
+
+  @Override
+  protected void execute(ScmClient scmClient) throws IOException {
+    try (ScmClient client = new ContainerOperationClient(

Review Comment:
   Thanks @sadanand48, made the required changes!



-- 
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] tanvipenumudy commented on pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on PR #4747:
URL: https://github.com/apache/ozone/pull/4747#issuecomment-1602455517

   Thank you @duongkame, made the required changes to avoid exposing `TimeoutException`.


-- 
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] duongkame commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "duongkame (via GitHub)" <gi...@apache.org>.
duongkame commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1203459147


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java:
##########
@@ -511,6 +512,18 @@ public static SCMSecurityProtocol getScmSecurityClient(
         SCMSecurityProtocol.class, conf);
   }
 
+  public static SecretKeyProtocolScm getScmSecretClient(

Review Comment:
   Add a javadoc to mention it's intended to be used by clients under SCM identify. Maybe rename it to `getSecretKeyClientForSCM` to make the intention clear.



##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java:
##########
@@ -362,6 +363,15 @@ StartContainerBalancerResponseProto startContainerBalancer(
    */
   List<String> getScmRatisRoles() throws IOException;
 
+  /**
+   * Force generates new secret keys (rotate).
+   *
+   * @param force boolean flag that forcefully rotates the key on demand
+   * @return
+   * @throws TimeoutException
+   */
+  boolean checkAndRotate(boolean force) throws TimeoutException, IOException;

Review Comment:
   nit: maybe name it `rotateSecretKeys`. Note that this class name has no SecretKey context, so the method name needs it.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/SecretKeyProtocolServerSideTranslatorPB.java:
##########
@@ -102,6 +104,16 @@ public SCMSecretKeyResponse processRequest(SCMSecretKeyRequest request)
             .setSecretKeysListResponseProto(getAllSecretKeys())
             .build();
 
+      case GetCheckAndRotate:
+        try {
+          return scmSecurityResponse
+              .setCheckAndRotateResponseProto(
+                  checkAndRotate(request.getCheckAndRotateRequest().getForce()))
+              .build();
+        } catch (TimeoutException e) {
+          LOG.error("Timeout occurred while executing checkAndRotate.", e);

Review Comment:
   See the exception handling several lines down. I think this timeout error can be considered an internal error and get transferred to client side. 



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokens.java:
##########
@@ -377,6 +386,23 @@ private static void setSecureConfig() throws IOException {
         ozoneKeytab.getAbsolutePath());
   }
 
+  @Test
+  public void testRotateKeySCMAdminCommand()

Review Comment:
   I think we need to move this test to a new test class. `TestBlockToken` is intended for block token test-cases and configure a relatively short secret key lifetime. This CLI test should be done with a longer rotation duration, e.g. 1h, so that the CLI and the automatic rotation don't collide. 



##########
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java:
##########
@@ -124,6 +129,16 @@ public static StorageContainerLocationProtocol newContainerRpcClient(
     return HAUtils.getScmContainerClient(configSource);
   }
 
+  public static SecretKeyProtocolScm newSecretKeyClient(
+      ConfigurationSource configSource) {
+    try {
+      return HddsServerUtil.getScmSecretClient(configSource);
+    } catch (IOException e) {
+      LOG.error("Error while getting the SCM secret client.", e);

Review Comment:
   Just throw, let the client code (the CLI catch and log it).



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java:
##########
@@ -232,6 +233,16 @@ private void validateSecretKeyStatus() throws SCMSecretKeyException {
     }
   }
 
+  @Override
+  public boolean checkAndRotate(boolean force) throws TimeoutException {
+    try {
+      validateSecretKeyStatus();
+    } catch (SCMSecretKeyException e) {
+      LOGGER.error("Error while validating the secret key status.", e);

Review Comment:
   Don't catch these validation exceptions on server side. We should let the translator to translate them to error code and transfer to client side.



-- 
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] tanvipenumudy commented on pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on PR #4747:
URL: https://github.com/apache/ozone/pull/4747#issuecomment-1561242757

   Thank you @adoroszlai, noted. Will be cancelling all the previous runs to enable quicker Git CI/CD checks over the latest commits.


-- 
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] tanvipenumudy commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1259153834


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeyManager.java:
##########
@@ -126,6 +127,8 @@ public synchronized boolean checkAndRotate() throws SCMException {
       state.updateKeys(updatedKeys);
       return true;
     }
+    LOG.info("The latest key was created at: " + currentKey.getCreationTime() +

Review Comment:
   Understood, will remove this log message from INFO, 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] tanvipenumudy commented on pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on PR #4747:
URL: https://github.com/apache/ozone/pull/4747#issuecomment-1558817693

   > I think the CLI should not rely on the admin to find the destination SCM, instead itself find the leader using the failover proxy, and only the leader SCM should accept this call, and process this call.
   
   The command should work even by skipping the `--scm` flag.
   
   > Also, the CLI can be divided into 2 usages.
   
   Sure, will be making the changes for adding an explicit `--force` flag to the SCM CLI.


-- 
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] adoroszlai commented on pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "adoroszlai (via GitHub)" <gi...@apache.org>.
adoroszlai commented on PR #4747:
URL: https://github.com/apache/ozone/pull/4747#issuecomment-1561155703

   @tanvipenumudy CI runs in forks for each push.  There is some limit on the concurrent Github Actions jobs/workflows for each repo (although I don't know the details).  If you push in multiple batches, you might want to cancel [runs](https://github.com/tanvipenumudy/ozone/actions/workflows/post-commit.yml?query=branch%3AHDDS-8659) for outdated versions of your code, otherwise the latest commits will be queued for a long time.


-- 
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] tanvipenumudy commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1204199905


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/RotateKeySubCommand.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.ozone.admin.scm;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+
+/**
+ * Handler of ozone admin scm rotate command.
+ */
+@CommandLine.Command(
+    name = "rotate",
+    description = "CLI command to force generate new keys (rotate)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class RotateKeySubCommand extends ScmSubcommand {
+
+  @CommandLine.ParentCommand
+  private ScmAdmin parent;
+
+  @Override
+  protected void execute(ScmClient scmClient) throws IOException {
+    try (ScmClient client = new ContainerOperationClient(
+        parent.getParent().getOzoneConf())) {
+      boolean status = false;
+      try {
+        status = client.checkAndRotate(true);
+      } catch (TimeoutException e) {
+        e.printStackTrace();
+      }
+      System.out.println(
+          "Secret key rotation is complete, a new key has been generated. " +
+              "Rotate Status: " + status);

Review Comment:
   Thank you for pointing this, made the required changes!



-- 
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] tanvipenumudy commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1202253466


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/protocolPB/SecretKeyProtocolClientSideTranslatorPB.java:
##########
@@ -50,7 +52,7 @@
  * {@link SecretKeyProtocol} to the server proxy.
  */
 public class SecretKeyProtocolClientSideTranslatorPB implements
-    SecretKeyProtocol, ProtocolTranslator, Closeable {
+    SecretKeyProtocol, SecretKeyProtocolScm, ProtocolTranslator, Closeable {

Review Comment:
   Thank you, I have made the changes to avoid redundant implementation.



-- 
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] krishnaasawa1 commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "krishnaasawa1 (via GitHub)" <gi...@apache.org>.
krishnaasawa1 commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1201936689


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/protocolPB/SecretKeyProtocolClientSideTranslatorPB.java:
##########
@@ -50,7 +52,7 @@
  * {@link SecretKeyProtocol} to the server proxy.
  */
 public class SecretKeyProtocolClientSideTranslatorPB implements
-    SecretKeyProtocol, ProtocolTranslator, Closeable {
+    SecretKeyProtocol, SecretKeyProtocolScm, ProtocolTranslator, Closeable {

Review Comment:
   Cosmetic SecretKeyProtocolScm extends SecretKeyProtocol. Not sure you need to have  SecretKeyProtocol here, if already implementing SecretKeyProtocolScm.



-- 
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] sadanand48 merged pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "sadanand48 (via GitHub)" <gi...@apache.org>.
sadanand48 merged PR #4747:
URL: https://github.com/apache/ozone/pull/4747


-- 
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] tanvipenumudy commented on pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on PR #4747:
URL: https://github.com/apache/ozone/pull/4747#issuecomment-1554560463

   @duongkame, @kerneltime, @sadanand48, @siddhantsangwan could you please take a look? 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] sadanand48 commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "sadanand48 (via GitHub)" <gi...@apache.org>.
sadanand48 commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1199195624


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/RotateKeySubCommand.java:
##########
@@ -0,0 +1,40 @@
+package org.apache.hadoop.ozone.admin.scm;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+
+/**
+ * Handler of ozone admin scm rotate command.
+ */
+@CommandLine.Command(
+    name = "rotate",
+    description = "CLI command to force generate new keys (rotate)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class RotateKeySubCommand extends ScmSubcommand {
+
+  @CommandLine.ParentCommand
+  private ScmAdmin parent;
+
+  @Override
+  protected void execute(ScmClient scmClient) throws IOException {
+    try (ScmClient client = new ContainerOperationClient(

Review Comment:
   We can use the same scmClient here from the method , for the unit test to work , we could just pass a list  of config args in the args array in --set=key=value format.



##########
hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java:
##########
@@ -124,6 +129,16 @@ public static StorageContainerLocationProtocol newContainerRpcClient(
     return HAUtils.getScmContainerClient(configSource);
   }
 
+  public static SecretKeyProtocolScm newSecretKeyClient(
+      ConfigurationSource configSource) {
+    try {
+      return HddsServerUtil.getScmSecretClient(configSource);
+    } catch (IOException e) {
+      e.printStackTrace();

Review Comment:
   LOG.error instead



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestBlockTokens.java:
##########
@@ -377,6 +386,24 @@ private static void setSecureConfig() throws IOException {
         ozoneKeytab.getAbsolutePath());
   }
 
+  @Test
+  public void testRotateKeySCMAdminCommand()
+      throws InterruptedException, TimeoutException, IOException {
+    GenericTestUtils.waitFor(() -> cluster.getScmLeader() != null, 100, 1000);
+    InetSocketAddress address =
+        cluster.getScmLeader().getClientRpcAddress();
+    String hostPort = address.getHostName() + ":" + address.getPort();
+    String[] args = {"scm", "rotate", "--scm", hostPort};
+
+    String oldKey =
+        scmClient.getSecretKeyClient().getCurrentSecretKey().toString();
+    Thread.sleep(1000);

Review Comment:
   Since the force flag rotates the key irrespective of whether duration has expired, this sleep might not be needed.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/SecretKeyProtocolServerSideTranslatorPB.java:
##########
@@ -102,6 +108,16 @@ public SCMSecretKeyResponse processRequest(SCMSecretKeyRequest request)
             .setSecretKeysListResponseProto(getAllSecretKeys())
             .build();
 
+      case GetCheckAndRotate:
+        try {
+          return scmSecurityResponse
+              .setCheckAndRotateResponseProto(
+                  checkAndRotate(request.getCheckAndRotateRequest().getForce()))
+              .build();
+        } catch (TimeoutException e) {
+          e.printStackTrace();

Review Comment:
   LOG.error instead



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java:
##########
@@ -232,6 +234,16 @@ private void validateSecretKeyStatus() throws SCMSecretKeyException {
     }
   }
 
+  @Override
+  public boolean checkAndRotate(boolean force) throws TimeoutException {
+    try {
+      validateSecretKeyStatus();
+    } catch (SCMSecretKeyException e) {
+      e.printStackTrace();

Review Comment:
   LOG.error instead



-- 
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] sadanand48 commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "sadanand48 (via GitHub)" <gi...@apache.org>.
sadanand48 commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1199195624


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/scm/RotateKeySubCommand.java:
##########
@@ -0,0 +1,40 @@
+package org.apache.hadoop.ozone.admin.scm;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+
+/**
+ * Handler of ozone admin scm rotate command.
+ */
+@CommandLine.Command(
+    name = "rotate",
+    description = "CLI command to force generate new keys (rotate)",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class RotateKeySubCommand extends ScmSubcommand {
+
+  @CommandLine.ParentCommand
+  private ScmAdmin parent;
+
+  @Override
+  protected void execute(ScmClient scmClient) throws IOException {
+    try (ScmClient client = new ContainerOperationClient(

Review Comment:
   We can use the same scmClient here from the method. For the unit test to work , we could just pass a list  of config args in the args array in --set=key=value format.



-- 
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] duongkame commented on pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "duongkame (via GitHub)" <gi...@apache.org>.
duongkame commented on PR #4747:
URL: https://github.com/apache/ozone/pull/4747#issuecomment-1555342686

   > Format: ozone admin scm rotate --scm hostPort
   
   I think the CLI should not rely on the admin to find the destination SCM, instead itself find the leader using the failover proxy, and only the leader SCM should accept this call, and process this call.
   
   Also, the CLI can be divided into 2 usages.
   
   ```
   ozone admin scm rotate
   # this command only trigger a rotation if the the latest secret key timestamp already pass the rotate duration
   # otherwise, it prints out a message to explain why rotation doesn't happen, e.g. `The lastest key was created at {lastest key timestamp} which doesn't pass the rotate duration of 1 day yet`.
   ```
   
   ```
   ozone admin scm rotate --force
   # force key rotation to happen 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.

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] tanvipenumudy commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1238377226


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/SecretKeyProtocolServerSideTranslatorPB.java:
##########
@@ -102,6 +104,21 @@ public SCMSecretKeyResponse processRequest(SCMSecretKeyRequest request)
             .setSecretKeysListResponseProto(getAllSecretKeys())
             .build();
 
+      case GetCheckAndRotate:
+        try {
+          return scmSecurityResponse
+              .setCheckAndRotateResponseProto(
+                  checkAndRotate(request.getCheckAndRotateRequest().getForce()))
+              .build();
+        } catch (TimeoutException e) {

Review Comment:
   Thanks @duongkame, made the required changes!



##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/protocol/SecretKeyProtocolScm.java:
##########
@@ -28,4 +31,14 @@
     clientPrincipal = HDDS_SCM_KERBEROS_PRINCIPAL_KEY
 )
 public interface SecretKeyProtocolScm extends SecretKeyProtocol {
+
+  /**
+   * Force generates new secret keys (rotate).
+   *
+   * @param force boolean flag that forcefully rotates the key on demand
+   * @return key rotation status
+   * @throws TimeoutException
+   * @throws IOException
+   */
+  boolean checkAndRotate(boolean force) throws TimeoutException, IOException;

Review Comment:
   Understood, thanks @duongkame, made the required changes!



-- 
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] tanvipenumudy commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1259137871


##########
hadoop-hdds/interface-server/src/main/proto/ScmSecretKeyProtocol.proto:
##########
@@ -67,12 +69,15 @@ message SCMSecretKeyResponse {
 
     optional SCMSecretKeysListResponse secretKeysListResponseProto = 13;
 
+    optional SCMGetCheckAndRotateResponse checkAndRotateResponseProto = 14;
+
 }
 
 enum Type {
     GetCurrentSecretKey = 1;
     GetSecretKey = 2;
     GetAllSecretKeys = 3;
+    GetCheckAndRotate = 4;

Review Comment:
   Thanks, I have renamed the field!



-- 
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] duongkame commented on a diff in pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "duongkame (via GitHub)" <gi...@apache.org>.
duongkame commented on code in PR #4747:
URL: https://github.com/apache/ozone/pull/4747#discussion_r1258918114


##########
hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeyManager.java:
##########
@@ -109,12 +109,13 @@ public boolean isInitialized() {
    *
    * @return true if rotation actually happens, false if it doesn't.
    */
-  public synchronized boolean checkAndRotate() throws SCMException {
+  public synchronized boolean checkAndRotate(boolean force)
+      throws SCMException {
     // Initialize the state if it's not initialized already.
     checkAndInitialize();
 
     ManagedSecretKey currentKey = state.getCurrentKey();
-    if (shouldRotate(currentKey)) {
+    if (force || shouldRotate(currentKey)) {

Review Comment:
   If the rotate is a result of a force, it'll be useful to add a log line to state that.



##########
hadoop-hdds/interface-server/src/main/proto/ScmSecretKeyProtocol.proto:
##########
@@ -67,12 +69,15 @@ message SCMSecretKeyResponse {
 
     optional SCMSecretKeysListResponse secretKeysListResponseProto = 13;
 
+    optional SCMGetCheckAndRotateResponse checkAndRotateResponseProto = 14;
+
 }
 
 enum Type {
     GetCurrentSecretKey = 1;
     GetSecretKey = 2;
     GetAllSecretKeys = 3;
+    GetCheckAndRotate = 4;

Review Comment:
   nit: name `CheckAndRotate` instead



-- 
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] tanvipenumudy commented on pull request #4747: HDDS-8659. Ozone admin SCM CLI command for block tokens

Posted by "tanvipenumudy (via GitHub)" <gi...@apache.org>.
tanvipenumudy commented on PR #4747:
URL: https://github.com/apache/ozone/pull/4747#issuecomment-1571973937

   Thank you for the reviews, I have addressed the comments on the PR. 
   
   The [Git CI: integration (ozone)](https://github.com/tanvipenumudy/ozone/actions/workflows/post-commit.yml?query=branch%3AHDDS-8659) seems to fail at `org.apache.hadoop.ozone.TestSecretKeysApi` - but the test cases have been passing locally.


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