You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by su...@apache.org on 2021/03/25 06:16:50 UTC

[hbase] branch branch-2.3 updated: HBASE-25562 ReplicationSourceWALReader log and handle exception immediately without retrying (#2966)

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

sunxin pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new 754caba  HBASE-25562 ReplicationSourceWALReader log and handle exception immediately without retrying (#2966)
754caba is described below

commit 754caba4796d24fbc5a0eae1d8205f601d78a961
Author: XinSun <dd...@gmail.com>
AuthorDate: Thu Mar 25 14:16:21 2021 +0800

    HBASE-25562 ReplicationSourceWALReader log and handle exception immediately without retrying (#2966)
    
    Signed-off-by: sandeepvinayak
---
 .../regionserver/ReplicationSourceWALReader.java   | 30 +++++++++++++---------
 1 file changed, 18 insertions(+), 12 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index 4a32962..d3c44a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -147,14 +147,15 @@ class ReplicationSourceWALReader extends Thread {
           }
         }
       } catch (IOException e) { // stream related
-        if (sleepMultiplier < maxRetriesMultiplier) {
-          LOG.debug("Failed to read stream of replication entries: " + e);
-          sleepMultiplier++;
+        if (handleEofException(e)) {
+          sleepMultiplier = 1;
         } else {
-          LOG.error("Failed to read stream of replication entries", e);
-          handleEofException(e);
+          LOG.warn("Failed to read stream of replication entries", e);
+          if (sleepMultiplier < maxRetriesMultiplier) {
+            sleepMultiplier ++;
+          }
+          Threads.sleep(sleepForRetries * sleepMultiplier);
         }
-        Threads.sleep(sleepForRetries * sleepMultiplier);
       } catch (InterruptedException e) {
         LOG.trace("Interrupted while sleeping between WAL reads");
         Thread.currentThread().interrupt();
@@ -241,24 +242,29 @@ class ReplicationSourceWALReader extends Thread {
     }
   }
 
-  // if we get an EOF due to a zero-length log, and there are other logs in queue
-  // (highly likely we've closed the current log), we've hit the max retries, and autorecovery is
-  // enabled, then dump the log
-  private void handleEofException(IOException e) {
+  /**
+   * if we get an EOF due to a zero-length log, and there are other logs in queue
+   * (highly likely we've closed the current log), and autorecovery is
+   * enabled, then dump the log
+   * @return true only the IOE can be handled
+   */
+  private boolean handleEofException(IOException e) {
     // Dump the log even if logQueue size is 1 if the source is from recovered Source
     // since we don't add current log to recovered source queue so it is safe to remove.
     if ((e instanceof EOFException || e.getCause() instanceof EOFException) &&
       (source.isRecovered() || logQueue.size() > 1) && this.eofAutoRecovery) {
       try {
         if (fs.getFileStatus(logQueue.peek()).getLen() == 0) {
-          LOG.warn("Forcing removal of 0 length log in queue: " + logQueue.peek());
+          LOG.warn("Forcing removal of 0 length log in queue: {}", logQueue.peek());
           logQueue.remove();
           currentPosition = 0;
+          return true;
         }
       } catch (IOException ioe) {
-        LOG.warn("Couldn't get file length information about log " + logQueue.peek());
+        LOG.warn("Couldn't get file length information about log {}", logQueue.peek());
       }
     }
+    return false;
   }
 
   public Path getCurrentPath() {