This is an automated email from the ASF dual-hosted git repository.
xucang pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-1 by this push:
new 3cff107 HBASE-25848: Add flexibility to backup replication in case
replication filter throws an exception (#3291)
3cff107 is described below
commit 3cff107a5f4eb638ca0126b556aae5b2afb02ae3
Author: Sandeep Pal <[email protected]>
AuthorDate: Fri May 21 17:22:13 2021 -0700
HBASE-25848: Add flexibility to backup replication in case replication
filter throws an exception (#3291)
---
.../ReplicationSourceWALReaderThread.java | 9 ++--
.../WALEntryFilterRetryableException.java | 40 ++++++++++++++
.../regionserver/TestWALEntryStream.java | 61 ++++++++++++++++++++++
3 files changed, 107 insertions(+), 3 deletions(-)
diff --git
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
index a1d64ca..54ed3ab 100644
---
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
+++
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReaderThread.java
@@ -181,15 +181,18 @@ public class ReplicationSourceWALReaderThread extends
Thread {
sleepMultiplier = 1;
}
}
- } catch (IOException | WALEntryStreamRuntimeException e) { // stream
related
+ } catch (IOException | WALEntryFilterRetryableException
+ | WALEntryStreamRuntimeException e) { // stream related
if (handleEofException(e, entryStream, batch)) {
sleepMultiplier = 1;
} else {
if (sleepMultiplier < maxRetriesMultiplier) {
- LOG.debug("Failed to read stream of replication entries: " + e);
+ LOG.debug("Failed to read stream of replication entries "
+ + "or replication filter is recovering " + e);
sleepMultiplier++;
} else {
- LOG.error("Failed to read stream of replication entries", e);
+ LOG.error("Failed to read stream of replication entries "
+ + "or replication filter is recovering " + e);
}
Threads.sleep(sleepForRetries * sleepMultiplier);
}
diff --git
a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryFilterRetryableException.java
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryFilterRetryableException.java
new file mode 100644
index 0000000..d0b7e15
--- /dev/null
+++
b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryFilterRetryableException.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * This exception should be thrown from any wal filter when the filter is
expected
+ * to recover from the failures and it wants the replication to backup till it
fails.
+ * There is special handling in replication wal reader to catch this exception
and
+ * retry.
+ */
[email protected](HBaseInterfaceAudience.REPLICATION)
+public class WALEntryFilterRetryableException extends RuntimeException {
+ private static final long serialVersionUID = 1L;
+
+ public WALEntryFilterRetryableException(String m, Throwable t) {
+ super(m, t);
+ }
+
+ public WALEntryFilterRetryableException(String m) {
+ super(m);
+ }
+}
diff --git
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
index 03c8d8a..6b98204 100644
---
a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
+++
b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
@@ -404,6 +404,41 @@ public class TestWALEntryStream {
}
@Test
+ public void testReplicationSourceWALReaderWithFailingFilter() throws
Exception {
+ appendEntriesToLog(3);
+ // get ending position
+ long position;
+ try (WALEntryStream entryStream =
+ new WALEntryStream(logQueue, fs, conf, new MetricsSource("1"),
fakeWalGroupId)) {
+ entryStream.next();
+ entryStream.next();
+ entryStream.next();
+ position = entryStream.getPosition();
+ }
+
+ int numFailuresInFilter = 5;
+ ReplicationSourceManager mockSourceManager =
Mockito.mock(ReplicationSourceManager.class);
+ ReplicationSource source = Mockito.mock(ReplicationSource.class);
+ when(source.isPeerEnabled()).thenReturn(true);
+ when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
+ ReplicationSourceWALReaderThread batcher =
+ new ReplicationSourceWALReaderThread(mockSourceManager,
getQueueInfo(),logQueue, 0,
+ fs, conf, getIntermittentFailingFilter(numFailuresInFilter), new
MetricsSource("1"),
+ source, fakeWalGroupId);
+ Path walPath = getQueue().peek();
+ batcher.start();
+ WALEntryBatch entryBatch = batcher.take();
+ assertEquals(numFailuresInFilter, FailingWALEntryFilter.numFailures());
+
+ // should've batched up our entries
+ assertNotNull(entryBatch);
+ assertEquals(3, entryBatch.getWalEntries().size());
+ assertEquals(position, entryBatch.getLastWalPosition());
+ assertEquals(walPath, entryBatch.getLastWalPath());
+ assertEquals(3, entryBatch.getNbRowKeys());
+ }
+
+ @Test
public void testReplicationSourceWALReaderThreadRecoveredQueue() throws
Exception {
appendEntriesToLog(3);
log.rollWriter();
@@ -619,6 +654,32 @@ public class TestWALEntryStream {
return edit;
}
+ private WALEntryFilter getIntermittentFailingFilter(int numFailuresInFilter)
{
+ return new FailingWALEntryFilter(numFailuresInFilter);
+ }
+
+ public static class FailingWALEntryFilter implements WALEntryFilter {
+ private int numFailures = 0;
+ private static int countFailures = 0;
+
+ public FailingWALEntryFilter(int numFailuresInFilter) {
+ numFailures = numFailuresInFilter;
+ }
+
+ @Override
+ public Entry filter(Entry entry) {
+ if (countFailures == numFailures) {
+ return entry;
+ }
+ countFailures = countFailures + 1;
+ throw new WALEntryFilterRetryableException("failing filter");
+ }
+
+ public static int numFailures(){
+ return countFailures;
+ }
+ }
+
private WALEntryFilter getDummyFilter() {
return new WALEntryFilter() {