Skip to content

Commit 2042523

Browse files
authored
HBASE-24877 Add option to avoid aborting RS process upon uncaught exc… (#2255)
Signed-off-by: Duo Zhang <zhangduo@apache.org> Signed-off-by: Josh Elser <elserj@apache.org>
1 parent dcba493 commit 2042523

File tree

3 files changed

+157
-13
lines changed

3 files changed

+157
-13
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java

Lines changed: 63 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -35,9 +35,12 @@
3535
import java.util.concurrent.PriorityBlockingQueue;
3636
import java.util.concurrent.TimeUnit;
3737
import java.util.concurrent.TimeoutException;
38+
import java.util.concurrent.atomic.AtomicBoolean;
3839
import java.util.concurrent.atomic.AtomicLong;
3940
import java.util.function.Predicate;
41+
4042
import org.apache.commons.lang3.StringUtils;
43+
import org.apache.commons.lang3.mutable.MutableBoolean;
4144
import org.apache.hadoop.conf.Configuration;
4245
import org.apache.hadoop.fs.FileSystem;
4346
import org.apache.hadoop.fs.Path;
@@ -120,6 +123,14 @@ public class ReplicationSource implements ReplicationSourceInterface {
120123
// ReplicationEndpoint which will handle the actual replication
121124
private volatile ReplicationEndpoint replicationEndpoint;
122125

126+
private boolean abortOnError;
127+
//This is needed for the startup loop to identify when there's already
128+
//an initialization happening (but not finished yet),
129+
//so that it doesn't try submit another initialize thread.
130+
//NOTE: this should only be set to false at the end of initialize method, prior to return.
131+
private AtomicBoolean startupOngoing = new AtomicBoolean(false);
132+
133+
123134
/**
124135
* A filter (or a chain of filters) for WAL entries; filters out edits.
125136
*/
@@ -217,6 +228,10 @@ public void init(Configuration conf, FileSystem fs, ReplicationSourceManager man
217228
this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0);
218229
this.totalBufferUsed = manager.getTotalBufferUsed();
219230
this.walFileLengthProvider = walFileLengthProvider;
231+
232+
this.abortOnError = this.conf.getBoolean("replication.source.regionserver.abort",
233+
true);
234+
220235
LOG.info("queueId={}, ReplicationSource: {}, currentBandwidth={}", queueId,
221236
replicationPeer.getId(), this.currentBandwidth);
222237
}
@@ -372,10 +387,10 @@ private void tryStartNewShipper(String walGroupId, PriorityBlockingQueue<Path> q
372387
createNewWALReader(walGroupId, queue, worker.getStartPosition());
373388
Threads.setDaemonThreadRunning(
374389
walReader, Thread.currentThread().getName()
375-
+ ".replicationSource.wal-reader." + walGroupId + "," + queueId,
376-
this::uncaughtException);
390+
+ ".replicationSource.wal-reader." + walGroupId + "," + queueId,
391+
(t,e) -> this.uncaughtException(t, e, this.manager, this.getPeerId()));
377392
worker.setWALReader(walReader);
378-
worker.startup(this::uncaughtException);
393+
worker.startup((t,e) -> this.uncaughtException(t, e, this.manager, this.getPeerId()));
379394
return worker;
380395
}
381396
});
@@ -450,11 +465,28 @@ WALEntryFilter getWalEntryFilter() {
450465
return walEntryFilter;
451466
}
452467

453-
protected final void uncaughtException(Thread t, Throwable e) {
468+
protected final void uncaughtException(Thread t, Throwable e,
469+
ReplicationSourceManager manager, String peerId) {
454470
RSRpcServices.exitIfOOME(e);
455471
LOG.error("Unexpected exception in {} currentPath={}",
456472
t.getName(), getCurrentPath(), e);
457-
server.abort("Unexpected exception in " + t.getName(), e);
473+
if(abortOnError){
474+
server.abort("Unexpected exception in " + t.getName(), e);
475+
}
476+
if(manager != null){
477+
while (true) {
478+
try {
479+
LOG.info("Refreshing replication sources now due to previous error on thread: {}",
480+
t.getName());
481+
manager.refreshSources(peerId);
482+
break;
483+
} catch (IOException e1) {
484+
LOG.error("Replication sources refresh failed.", e1);
485+
sleepForRetries("Sleeping before try refreshing sources again",
486+
maxRetriesMultiplier);
487+
}
488+
}
489+
}
458490
}
459491

460492
@Override
@@ -544,12 +576,16 @@ private void initialize() {
544576
replicationEndpoint.stop();
545577
if (sleepForRetries("Error starting ReplicationEndpoint", sleepMultiplier)) {
546578
sleepMultiplier++;
579+
} else {
580+
this.startupOngoing.set(false);
581+
throw new RuntimeException("Exhausted retries to start replication endpoint.");
547582
}
548583
}
549584
}
550585

551586
if (!this.isSourceActive()) {
552-
return;
587+
this.startupOngoing.set(false);
588+
throw new IllegalStateException("Source should be active.");
553589
}
554590

555591
sleepMultiplier = 1;
@@ -571,7 +607,8 @@ private void initialize() {
571607
}
572608

573609
if(!this.isSourceActive()) {
574-
return;
610+
this.startupOngoing.set(false);
611+
throw new IllegalStateException("Source should be active.");
575612
}
576613
LOG.info("{} Source: {}, is now replicating from cluster: {}; to peer cluster: {};",
577614
logPeerId(), this.replicationQueueInfo.getQueueId(), clusterId, peerClusterId);
@@ -583,16 +620,30 @@ private void initialize() {
583620
PriorityBlockingQueue<Path> queue = entry.getValue();
584621
tryStartNewShipper(walGroupId, queue);
585622
}
623+
this.startupOngoing.set(false);
586624
}
587625

588626
@Override
589627
public void startup() {
590-
// mark we are running now
628+
//Flag that signalizes uncaught error happening while starting up the source
629+
// and a retry should be attempted
630+
MutableBoolean retryStartup = new MutableBoolean(true);
591631
this.sourceRunning = true;
592-
initThread = new Thread(this::initialize);
593-
Threads.setDaemonThreadRunning(initThread,
594-
Thread.currentThread().getName() + ".replicationSource," + this.queueId,
595-
this::uncaughtException);
632+
do {
633+
if(retryStartup.booleanValue()) {
634+
retryStartup.setValue(false);
635+
startupOngoing.set(true);
636+
// mark we are running now
637+
initThread = new Thread(this::initialize);
638+
Threads.setDaemonThreadRunning(initThread,
639+
Thread.currentThread().getName() + ".replicationSource," + this.queueId,
640+
(t,e) -> {
641+
sourceRunning = false;
642+
uncaughtException(t, e, null, null);
643+
retryStartup.setValue(!this.abortOnError);
644+
});
645+
}
646+
} while (this.startupOngoing.get() && !this.abortOnError);
596647
}
597648

598649
@Override

hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -290,7 +290,8 @@ private boolean updateLogPosition(WALEntryBatch batch) {
290290
public void startup(UncaughtExceptionHandler handler) {
291291
String name = Thread.currentThread().getName();
292292
Threads.setDaemonThreadRunning(this,
293-
name + ".replicationSource.shipper" + walGroupId + "," + source.getQueueId(), handler);
293+
name + ".replicationSource.shipper" + walGroupId + "," + source.getQueueId(),
294+
handler::uncaughtException);
294295
}
295296

296297
Path getCurrentPath() {

hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java

Lines changed: 92 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
package org.apache.hadoop.hbase.replication.regionserver;
1919
import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.META_WAL_PROVIDER_ID;
2020
import static org.junit.Assert.assertEquals;
21+
import static org.junit.Assert.assertFalse;
2122
import static org.junit.Assert.assertNotNull;
2223
import static org.junit.Assert.assertNull;
2324
import static org.junit.Assert.assertTrue;
@@ -396,6 +397,25 @@ protected void doStop() {
396397
}
397398
}
398399

400+
/**
401+
* Deadend Endpoint. Does nothing.
402+
*/
403+
public static class FaultyReplicationEndpoint extends DoNothingReplicationEndpoint {
404+
405+
static int count = 0;
406+
407+
@Override
408+
public synchronized UUID getPeerUUID() {
409+
if(count==0) {
410+
count++;
411+
throw new RuntimeException();
412+
} else {
413+
return super.getPeerUUID();
414+
}
415+
}
416+
417+
}
418+
399419
/**
400420
* Test HBASE-20497
401421
* Moved here from TestReplicationSource because doesn't need cluster.
@@ -423,5 +443,77 @@ public void testRecoveredReplicationSourceShipperGetPosition() throws Exception
423443
new RecoveredReplicationSourceShipper(conf, walGroupId, queue, source, storage);
424444
assertEquals(1001L, shipper.getStartPosition());
425445
}
446+
447+
/**
448+
* Test ReplicationSource retries startup once an uncaught exception happens
449+
* during initialization and <b>eplication.source.regionserver.abort</b> is set to false.
450+
*/
451+
@Test
452+
public void testAbortFalseOnError() throws IOException {
453+
ReplicationSource rs = new ReplicationSource();
454+
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
455+
conf.setInt("replication.source.maxretriesmultiplier", 1);
456+
conf.setBoolean("replication.source.regionserver.abort", false);
457+
ReplicationPeer mockPeer = Mockito.mock(ReplicationPeer.class);
458+
Mockito.when(mockPeer.getConfiguration()).thenReturn(conf);
459+
Mockito.when(mockPeer.getPeerBandwidth()).thenReturn(0L);
460+
ReplicationPeerConfig peerConfig = Mockito.mock(ReplicationPeerConfig.class);
461+
Mockito.when(peerConfig.getReplicationEndpointImpl()).
462+
thenReturn(FaultyReplicationEndpoint.class.getName());
463+
Mockito.when(mockPeer.getPeerConfig()).thenReturn(peerConfig);
464+
ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class);
465+
Mockito.when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong());
466+
String queueId = "qid";
467+
RegionServerServices rss =
468+
TEST_UTIL.createMockRegionServerService(ServerName.parseServerName("a.b.c,1,1"));
469+
rs.init(conf, null, manager, null, mockPeer, rss, queueId, null,
470+
p -> OptionalLong.empty(), new MetricsSource(queueId));
471+
try {
472+
rs.startup();
473+
assertTrue(rs.isSourceActive());
474+
assertEquals(0, rs.getSourceMetrics().getSizeOfLogQueue());
475+
rs.enqueueLog(new Path("a.1" + META_WAL_PROVIDER_ID));
476+
assertEquals(0, rs.getSourceMetrics().getSizeOfLogQueue());
477+
rs.enqueueLog(new Path("a.1"));
478+
assertEquals(1, rs.getSourceMetrics().getSizeOfLogQueue());
479+
} finally {
480+
rs.terminate("Done");
481+
rss.stop("Done");
482+
}
483+
}
484+
485+
/**
486+
* Test ReplicationSource retries startup once an uncaught exception happens
487+
* during initialization and <b>replication.source.regionserver.abort</b> is set to false.
488+
*/
489+
@Test
490+
public void testAbortTrueOnError() throws IOException {
491+
ReplicationSource rs = new ReplicationSource();
492+
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
493+
conf.setInt("replication.source.maxretriesmultiplier", 1);
494+
ReplicationPeer mockPeer = Mockito.mock(ReplicationPeer.class);
495+
Mockito.when(mockPeer.getConfiguration()).thenReturn(conf);
496+
Mockito.when(mockPeer.getPeerBandwidth()).thenReturn(0L);
497+
ReplicationPeerConfig peerConfig = Mockito.mock(ReplicationPeerConfig.class);
498+
Mockito.when(peerConfig.getReplicationEndpointImpl()).
499+
thenReturn(FaultyReplicationEndpoint.class.getName());
500+
Mockito.when(mockPeer.getPeerConfig()).thenReturn(peerConfig);
501+
ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class);
502+
Mockito.when(manager.getTotalBufferUsed()).thenReturn(new AtomicLong());
503+
String queueId = "qid";
504+
RegionServerServices rss =
505+
TEST_UTIL.createMockRegionServerService(ServerName.parseServerName("a.b.c,1,1"));
506+
rs.init(conf, null, manager, null, mockPeer, rss, queueId, null,
507+
p -> OptionalLong.empty(), new MetricsSource(queueId));
508+
try {
509+
rs.startup();
510+
Waiter.waitFor(conf, 1000, () -> FaultyReplicationEndpoint.count > 0);
511+
assertFalse(rs.isSourceActive());
512+
assertTrue(rss.isAborted());
513+
} finally {
514+
rs.terminate("Done");
515+
rss.stop("Done");
516+
}
517+
}
426518
}
427519

0 commit comments

Comments
 (0)