You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2020/12/02 18:04:03 UTC
[hbase] branch master updated: HBASE-25349 [Flakey Tests] branch-2
TestRefreshRecoveredReplication.testReplicationRefreshSource:141 Waiting
timed out after [60, 000] msec (#2731)
This is an automated email from the ASF dual-hosted git repository.
stack pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new 946fa81 HBASE-25349 [Flakey Tests] branch-2 TestRefreshRecoveredReplication.testReplicationRefreshSource:141 Waiting timed out after [60,000] msec (#2731)
946fa81 is described below
commit 946fa81715e8bd1d705bc07061533454bdfd1ae4
Author: Michael Stack <sa...@users.noreply.github.com>
AuthorDate: Wed Dec 2 09:55:24 2020 -0800
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 <nd...@apache.org>
---
.../regionserver/ReplicationSourceManager.java | 2 +-
.../TestRefreshRecoveredReplication.java | 29 ++++++++++++++--------
2 files changed, 20 insertions(+), 11 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 95157ca..c116680 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -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));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRefreshRecoveredReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRefreshRecoveredReplication.java
index f84f32a..cf4f710 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRefreshRecoveredReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRefreshRecoveredReplication.java
@@ -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);