[ 
https://issues.apache.org/jira/browse/HDFS-13603?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17849478#comment-17849478
 ] 

ASF GitHub Bot commented on HDFS-13603:
---------------------------------------

yzhang559 commented on code in PR #6774:
URL: https://github.com/apache/hadoop/pull/6774#discussion_r1614787242


##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java:
##########
@@ -269,12 +269,23 @@ public ValueQueue(final int numValues, final float 
lowWaterMark, long expiry,
    * Initializes the Value Queues for the provided keys by calling the
    * fill Method with "numInitValues" values
    * @param keyNames Array of key Names
-   * @throws ExecutionException executionException.
+   * @throws IOException if no successful initialization for any key
    */
-  public void initializeQueuesForKeys(String... keyNames)
-      throws ExecutionException {
+  public void initializeQueuesForKeys(String... keyNames) throws IOException {
+    int successfulInitializations = 0;
+    ExecutionException lastException = null;
+
     for (String keyName : keyNames) {
-      keyQueues.get(keyName);
+      try {
+        keyQueues.get(keyName);
+        successfulInitializations++;
+      } catch (ExecutionException e) {
+        lastException = e;
+      }
+    }
+
+    if (keyNames.length > 0 && successfulInitializations == 0) {
+      throw new IOException("Failed to initialize any queue for the provided 
keys.", lastException);

Review Comment:
   So this is used to indicate warmup should be retried. 
   Since the return value is void of this func and it's caller funcs, would 
rather keep the IOException. Otherwise, it's hard to update the `success` flag 
in 
[.../hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java](https://github.com/apache/hadoop/pull/6774/files/10d763a5ff514541ef1eea11d70bf5173374a5d1#diff-092663652ffe33b10e51bfa062724d70e6334a9a14e64db35c9854805e09da14)





> Warmup NameNode EDEK thread retries continuously if there's an invalid key 
> ---------------------------------------------------------------------------
>
>                 Key: HDFS-13603
>                 URL: https://issues.apache.org/jira/browse/HDFS-13603
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: encryption, namenode
>    Affects Versions: 2.8.0
>            Reporter: Antony Jay
>            Priority: Major
>              Labels: pull-request-available
>
> https://issues.apache.org/jira/browse/HDFS-9405 adds a background thread to 
> pre-warm EDEK cache. 
> However this fails and retries continuously if key retrieval fails for one 
> encryption zone. In our usecase, we have temporarily removed keys for certain 
> encryption zones.  Currently namenode and kms log is filled up with errors 
> related to background thread retrying warmup for ever .
> The pre-warm thread should
>  * Continue to refresh other encryption zones even if it fails for one
>  * Should retry only if it fails for all encryption zones, which will be the 
> case when kms is down.
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to