This is an automated email from the ASF dual-hosted git repository.

ritesh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 0fa2e559f9 HDDS-9020. Datanodes fails to start up when secret key has 
not yet been initialized in SCM. (#5068)
0fa2e559f9 is described below

commit 0fa2e559f9d48dd3f04b8f7c5378c6cfe4ad1889
Author: Duong Nguyen <[email protected]>
AuthorDate: Mon Jul 17 11:05:31 2023 -0700

    HDDS-9020. Datanodes fails to start up when secret key has not yet been 
initialized in SCM. (#5068)
---
 .../symmetric/DefaultSecretKeySignerClient.java    | 48 +++++++++++++++++++++-
 .../security/symmetric/SecretKeySignerClient.java  |  2 +-
 2 files changed, 47 insertions(+), 3 deletions(-)

diff --git 
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/DefaultSecretKeySignerClient.java
 
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/DefaultSecretKeySignerClient.java
index d5e12689f9..f9358a1422 100644
--- 
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/DefaultSecretKeySignerClient.java
+++ 
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/DefaultSecretKeySignerClient.java
@@ -20,6 +20,10 @@ package org.apache.hadoop.hdds.security.symmetric;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.SecretKeyProtocol;
+import org.apache.hadoop.hdds.security.exception.SCMSecretKeyException;
+import 
org.apache.hadoop.hdds.security.exception.SCMSecretKeyException.ErrorCode;
+import org.apache.hadoop.hdds.utils.RetriableTask;
+import org.apache.hadoop.io.retry.RetryPolicy;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,6 +38,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 
 import static java.util.Objects.requireNonNull;
+import static org.apache.hadoop.io.retry.RetryPolicies.exponentialBackoffRetry;
+import static org.apache.hadoop.io.retry.RetryPolicy.RetryAction.FAIL;
 
 /**
  * Default implementation of {@link SecretKeySignerClient} that fetches
@@ -68,13 +74,51 @@ public class DefaultSecretKeySignerClient implements 
SecretKeySignerClient {
 
   @Override
   public void start(ConfigurationSource conf) throws IOException {
-    final ManagedSecretKey initialKey =
-        secretKeyProtocol.getCurrentSecretKey();
+    final ManagedSecretKey initialKey = loadInitialSecretKey();
+
     LOG.info("Initial secret key fetched from SCM: {}.", initialKey);
     cache.set(initialKey);
     scheduleSecretKeyPoller(conf, initialKey.getCreationTime());
   }
 
+  private ManagedSecretKey loadInitialSecretKey() throws IOException {
+    // Load initial active secret key from SCM, retries with exponential
+    // backoff when SCM has not initialized secret keys yet.
+
+    // Exponential backoff policy, 100 max retries, exponential backoff
+    // wait time that repeats each 10. The wait times can be illustrated as:
+    // 1 2 4 8 ... 512 1 2 4 8 ... 512 1 2 ...
+    // Maximum total delay is around 200min.
+    int maxRetries = 100;
+    int backoffCircle = 10;
+    int baseWaitTime = 1;
+    final RetryPolicy expBackoff =
+        exponentialBackoffRetry(backoffCircle, baseWaitTime, TimeUnit.SECONDS);
+
+    RetryPolicy retryPolicy = (ex, retries, failovers, isIdempotent) -> {
+      if (ex instanceof SCMSecretKeyException) {
+        ErrorCode errorCode = ((SCMSecretKeyException) ex).getErrorCode();
+        if (errorCode == ErrorCode.SECRET_KEY_NOT_INITIALIZED
+            && retries < maxRetries) {
+          return expBackoff.shouldRetry(ex, retries % backoffCircle,
+              failovers, isIdempotent);
+        }
+      }
+      return FAIL;
+    };
+
+    RetriableTask<ManagedSecretKey> task = new RetriableTask<>(retryPolicy,
+        "getCurrentSecretKey", secretKeyProtocol::getCurrentSecretKey);
+    try {
+      return task.call();
+    } catch (IOException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new IllegalStateException(
+          "Unexpected exception getting current secret key", e);
+    }
+  }
+
   @Override
   public void stop() {
     if (executorService != null) {
diff --git 
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeySignerClient.java
 
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeySignerClient.java
index 0ae010545f..145d20d6d0 100644
--- 
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeySignerClient.java
+++ 
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/symmetric/SecretKeySignerClient.java
@@ -29,7 +29,7 @@ public interface SecretKeySignerClient {
   ManagedSecretKey getCurrentSecretKey();
 
   /**
-   * This is where the actual implementation can  prefetch the current
+   * This is where the actual implementation can prefetch the current
    * secret key or initialize ay necessary resources, e.g. cache or executors.
    */
   default void start(ConfigurationSource conf) throws IOException {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to