HBASE-25349 [Flakey Tests] branch-2 TestRefreshRecoveredReplication.testReplicationRefreshSource:141 Waiting timed out after [60,000] msec (#2731)

Start the check for recovered queue presence earlier.

Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
This commit is contained in:
Michael Stack 2020-12-02 09:55:24 -08:00 committed by stack
parent 56eb5c9fc8
commit 946fa81715
2 changed files with 20 additions and 11 deletions

View File

@ -999,7 +999,7 @@ public class ReplicationSourceManager implements ReplicationListener {
wals.add(wal);
}
oldsources.add(src);
LOG.trace("Added source for recovered queue: " + src.getQueueId());
LOG.info("Added source for recovered queue {}", src.getQueueId());
for (String wal : walsSet) {
LOG.trace("Enqueueing log from recovered queue for source: " + src.getQueueId());
src.enqueueLog(new Path(oldLogDir, wal));

View File

@ -18,7 +18,10 @@
package org.apache.hadoop.hbase.replication.regionserver;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
@ -32,6 +35,7 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.replication.TestReplicationBase;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@ -51,6 +55,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
import static org.junit.Assert.assertEquals;
/**
* Testcase for HBASE-24871.
@ -75,6 +80,7 @@ public class TestRefreshRecoveredReplication extends TestReplicationBase {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// NUM_SLAVES1 is presumed 2 in below.
NUM_SLAVES1 = 2;
// replicate slowly
Configuration conf1 = UTIL1.getConfiguration();
@ -121,22 +127,25 @@ public class TestRefreshRecoveredReplication extends TestReplicationBase {
table1.put(new Put(r).addColumn(famName, famName, r));
}
// kill rs holding table region
Optional<RegionServerThread> server = UTIL1.getMiniHBaseCluster().getLiveRegionServerThreads()
.stream()
// Kill rs holding table region. There are only TWO servers. We depend on it.
List<RegionServerThread> rss = UTIL1.getMiniHBaseCluster().getLiveRegionServerThreads();
assertEquals(2, rss.size());
Optional<RegionServerThread> server = rss.stream()
.filter(rst -> CollectionUtils.isNotEmpty(rst.getRegionServer().getRegions(tablename)))
.findAny();
Assert.assertTrue(server.isPresent());
HRegionServer otherServer = rss.get(0).getRegionServer() == server.get().getRegionServer()?
rss.get(1).getRegionServer(): rss.get(0).getRegionServer();
server.get().getRegionServer().abort("stopping for test");
// waiting for recovered peer to appear.
Replication replication = (Replication)otherServer.getReplicationSourceService();
UTIL1.waitFor(60000, () -> !replication.getReplicationManager().getOldSources().isEmpty());
// Wait on only one server being up.
UTIL1.waitFor(60000, () ->
UTIL1.getMiniHBaseCluster().getLiveRegionServerThreads().size() == NUM_SLAVES1 - 1);
// Have to go back to source here because getLiveRegionServerThreads makes new array each time
UTIL1.getMiniHBaseCluster().getLiveRegionServerThreads().size() == NUM_SLAVES1 - 1);
UTIL1.waitTableAvailable(tablename);
// waiting for recovered peer to start
Replication replication = (Replication) UTIL1.getMiniHBaseCluster()
.getLiveRegionServerThreads().get(0).getRegionServer().getReplicationSourceService();
UTIL1.waitFor(60000, () ->
!replication.getReplicationManager().getOldSources().isEmpty());
LOG.info("Available {}", tablename);
// disable peer to trigger refreshSources
hbaseAdmin.disableReplicationPeer(PEER_ID2);