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

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

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


##########
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java:
##########
@@ -580,15 +583,15 @@ public void run() {
       final int logCoolDown = 10000; // periodically print error log (if any)
       int sinceLastLog = logCoolDown; // always print the first failure
       boolean success = false;
+      int retryCount = 0;
       IOException lastSeenIOE = null;
       long warmUpEDEKStartTime = monotonicNow();
-      while (true) {
+
+      while (!success && retryCount < maxRetries) {
         try {
           kp.warmUpEncryptedKeys(keyNames);
-          NameNode.LOG
-              .info("Successfully warmed up {} EDEKs.", keyNames.length);
+          NameNode.LOG.info("Successfully warmed up {} EDEKs.", 
keyNames.length);
           success = true;
-          break;
         } catch (IOException ioe) {
           lastSeenIOE = ioe;
           if (sinceLastLog >= logCoolDown) {

Review Comment:
   good catch, remove them since log are bounded now





> 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