HBASE-25562 ReplicationSourceWALReader log and handle exception immediately without retrying (#2943)
Signed-off-by: Wellington Chevreuil <wchevreuil@apache.org>
Signed-off-by: stack <stack@apache.org>
Signed-off-by: shahrs87
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 ad06df2..301a9e8 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
@@ -150,14 +150,13 @@
}
}
} catch (IOException e) { // stream related
- if (sleepMultiplier < maxRetriesMultiplier) {
- LOG.debug("Failed to read stream of replication entries: " + e);
- sleepMultiplier++;
- } else {
- LOG.error("Failed to read stream of replication entries", e);
- handleEofException(e);
+ if (!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();
@@ -245,10 +244,13 @@
}
}
- // 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) {
PriorityBlockingQueue<Path> queue = logQueue.getQueue(walGroupId);
// 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.
@@ -256,14 +258,16 @@
(source.isRecovered() || queue.size() > 1) && this.eofAutoRecovery) {
try {
if (fs.getFileStatus(queue.peek()).getLen() == 0) {
- LOG.warn("Forcing removal of 0 length log in queue: " + queue.peek());
+ LOG.warn("Forcing removal of 0 length log in queue: {}", queue.peek());
logQueue.remove(walGroupId);
currentPosition = 0;
+ return true;
}
} catch (IOException ioe) {
- LOG.warn("Couldn't get file length information about log " + queue.peek());
+ LOG.warn("Couldn't get file length information about log {}", queue.peek());
}
}
+ return false;
}
public Path getCurrentPath() {