HBASE-20612 TestReplicationKillSlaveRSWithSeparateOldWALs sometimes fail because it uses an expired cluster conn

This commit is contained in:
huzheng 2018-05-22 19:39:28 +08:00
parent dace8ff2a4
commit 5721150c6d
2 changed files with 50 additions and 49 deletions

View File

@ -108,7 +108,6 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
} catch (PreemptiveFastFailException e) { } catch (PreemptiveFastFailException e) {
throw e; throw e;
} catch (Throwable t) { } catch (Throwable t) {
Throwable e = t.getCause();
ExceptionUtil.rethrowIfInterrupt(t); ExceptionUtil.rethrowIfInterrupt(t);
Throwable cause = t.getCause(); Throwable cause = t.getCause();
if (cause instanceof DoNotRetryIOException) { if (cause instanceof DoNotRetryIOException) {

View File

@ -22,9 +22,12 @@ import static org.junit.Assert.fail;
import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.junit.ClassRule; import org.junit.ClassRule;
@ -32,7 +35,7 @@ import org.junit.experimental.categories.Category;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@Category({ReplicationTests.class, LargeTests.class}) @Category({ ReplicationTests.class, LargeTests.class })
public class TestReplicationKillRS extends TestReplicationBase { public class TestReplicationKillRS extends TestReplicationBase {
@ClassRule @ClassRule
@ -42,38 +45,34 @@ public class TestReplicationKillRS extends TestReplicationBase {
private static final Logger LOG = LoggerFactory.getLogger(TestReplicationKillRS.class); private static final Logger LOG = LoggerFactory.getLogger(TestReplicationKillRS.class);
/** /**
* Load up 1 tables over 2 region servers and kill a source during * Load up 1 tables over 2 region servers and kill a source during the upload. The failover
* the upload. The failover happens internally. * happens internally. WARNING this test sometimes fails because of HBASE-3515
*
* WARNING this test sometimes fails because of HBASE-3515
*
* @throws Exception
*/ */
public void loadTableAndKillRS(HBaseTestingUtility util) throws Exception { public void loadTableAndKillRS(HBaseTestingUtility util) throws Exception {
// killing the RS with hbase:meta can result into failed puts until we solve // killing the RS with hbase:meta can result into failed puts until we solve
// IO fencing // IO fencing
int rsToKill1 = int rsToKill1 = util.getHBaseCluster().getServerWithMeta() == 0 ? 1 : 0;
util.getHBaseCluster().getServerWithMeta() == 0 ? 1 : 0;
// Takes about 20 secs to run the full loading, kill around the middle // Takes about 20 secs to run the full loading, kill around the middle
Thread killer = killARegionServer(util, 5000, rsToKill1); Thread killer = killARegionServer(util, 5000, rsToKill1);
LOG.info("Start loading table");
int initialCount = utility1.loadTable(htable1, famName);
LOG.info("Done loading table");
killer.join(5000);
LOG.info("Done waiting for threads");
Result[] res; Result[] res;
while (true) { int initialCount;
try { try (Connection conn = ConnectionFactory.createConnection(conf1)) {
Scan scan = new Scan(); try (Table table = conn.getTable(tableName)) {
ResultScanner scanner = htable1.getScanner(scan); LOG.info("Start loading table");
res = scanner.next(initialCount); initialCount = utility1.loadTable(table, famName);
scanner.close(); LOG.info("Done loading table");
break; killer.join(5000);
} catch (UnknownScannerException ex) { LOG.info("Done waiting for threads");
LOG.info("Cluster wasn't ready yet, restarting scanner");
while (true) {
try (ResultScanner scanner = table.getScanner(new Scan())) {
res = scanner.next(initialCount);
break;
} catch (UnknownScannerException ex) {
LOG.info("Cluster wasn't ready yet, restarting scanner");
}
}
} }
} }
// Test we actually have all the rows, we may miss some because we // Test we actually have all the rows, we may miss some because we
@ -85,36 +84,39 @@ public class TestReplicationKillRS extends TestReplicationBase {
} }
int lastCount = 0; int lastCount = 0;
final long start = System.currentTimeMillis(); final long start = System.currentTimeMillis();
int i = 0; int i = 0;
while (true) { try (Connection conn = ConnectionFactory.createConnection(conf2)) {
if (i==NB_RETRIES-1) { try (Table table = conn.getTable(tableName)) {
fail("Waited too much time for queueFailover replication. " + while (true) {
"Waited "+(System.currentTimeMillis() - start)+"ms."); if (i == NB_RETRIES - 1) {
} fail("Waited too much time for queueFailover replication. " + "Waited "
Scan scan2 = new Scan(); + (System.currentTimeMillis() - start) + "ms.");
ResultScanner scanner2 = htable2.getScanner(scan2); }
Result[] res2 = scanner2.next(initialCount * 2); Result[] res2;
scanner2.close(); try (ResultScanner scanner = table.getScanner(new Scan())) {
if (res2.length < initialCount) { res2 = scanner.next(initialCount * 2);
if (lastCount < res2.length) { }
i--; // Don't increment timeout if we make progress if (res2.length < initialCount) {
} else { if (lastCount < res2.length) {
i++; i--; // Don't increment timeout if we make progress
} else {
i++;
}
lastCount = res2.length;
LOG.info(
"Only got " + lastCount + " rows instead of " + initialCount + " current i=" + i);
Thread.sleep(SLEEP_TIME * 2);
} else {
break;
}
} }
lastCount = res2.length;
LOG.info("Only got " + lastCount + " rows instead of " +
initialCount + " current i=" + i);
Thread.sleep(SLEEP_TIME*2);
} else {
break;
} }
} }
} }
private static Thread killARegionServer(final HBaseTestingUtility utility, private static Thread killARegionServer(final HBaseTestingUtility utility, final long timeout,
final long timeout, final int rs) { final int rs) {
Thread killer = new Thread() { Thread killer = new Thread() {
@Override @Override
public void run() { public void run() {