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

ctubbsii pushed a commit to branch 2.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/2.1 by this push:
     new fa796c64cd Always use retry backoff in ZooReader(Writer) (#3720)
fa796c64cd is described below

commit fa796c64cd28dbc6e713b8c1e7283f63ba376834
Author: Christopher Tubbs <ctubb...@apache.org>
AuthorDate: Thu Aug 24 15:31:24 2023 -0400

    Always use retry backoff in ZooReader(Writer) (#3720)
    
    Always use the retry backoff when always retrying. This fixes an issue
    where an "always retry" condition is triggered, it will not barrage
    ZooKeeper with repeated immediate retries.
    
    This fixes #3718
---
 .../org/apache/accumulo/core/fate/zookeeper/ZooReader.java     | 10 ++++++++--
 .../accumulo/core/fate/zookeeper/ZooReaderWriterTest.java      |  3 +++
 2 files changed, 11 insertions(+), 2 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java 
b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java
index 05de7af365..26fb6efa83 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooReader.java
@@ -184,7 +184,12 @@ public class ZooReader {
       try {
         return zkf.apply(getZooKeeper());
       } catch (KeeperException e) {
-        if (alwaysRetryCondition.test(e) || useRetryForTransient(retries, e)) {
+        if (alwaysRetryCondition.test(e)) {
+          retries.waitForNextAttempt(log,
+              "attempting to communicate with zookeeper after exception that 
always requires retry: "
+                  + e.getMessage());
+          continue;
+        } else if (useRetryForTransient(retries, e)) {
           continue;
         }
         throw e;
@@ -201,7 +206,8 @@ public class ZooReader {
       log.warn("Saw (possibly) transient exception communicating with 
ZooKeeper", e);
       if (retries.canRetry()) {
         retries.useRetry();
-        retries.waitForNextAttempt(log, "attempting to communicate with 
zookeeper after exception");
+        retries.waitForNextAttempt(log,
+            "attempting to communicate with zookeeper after exception: " + 
e.getMessage());
         return true;
       }
       log.error("Retry attempts ({}) exceeded trying to communicate with 
ZooKeeper",
diff --git 
a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriterTest.java
 
b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriterTest.java
index 567ba0d81a..0379c413fe 100644
--- 
a/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriterTest.java
+++ 
b/core/src/test/java/org/apache/accumulo/core/fate/zookeeper/ZooReaderWriterTest.java
@@ -134,6 +134,9 @@ public class ZooReaderWriterTest {
     // Let 2nd setData succeed
     expect(zk.setData(path, mutatedBytes, 0)).andReturn(null);
 
+    retry.waitForNextAttempt(anyObject(), anyObject());
+    expectLastCall().once();
+
     replay(zk, zrw, retryFactory, retry);
 
     assertArrayEquals(new byte[] {1}, zrw.mutateOrCreate(path, value, 
mutator));

Reply via email to