HBASE-10249 Intermittent TestReplicationSyncUpTool failure
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1554367 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0277f56616
commit
ab5b219cce
|
@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||
|
@ -37,6 +39,8 @@ import org.junit.experimental.categories.Category;
|
|||
@Category(LargeTests.class)
|
||||
public class TestReplicationSyncUpTool extends TestReplicationBase {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestReplicationSyncUpTool.class);
|
||||
|
||||
private static final byte[] t1_su = Bytes.toBytes("t1_syncup");
|
||||
private static final byte[] t2_su = Bytes.toBytes("t2_syncup");
|
||||
|
||||
|
@ -294,14 +298,34 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
|
|||
assertEquals("@Peer1 t2_syncup should still have 200 rows", 200, rowCount_ht2TargetAtPeer1);
|
||||
|
||||
// After sync up
|
||||
syncUp(utility1);
|
||||
rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
|
||||
rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
|
||||
assertEquals("@Peer1 t1_syncup should be sync up and have 50 rows", 50,
|
||||
rowCount_ht1TargetAtPeer1);
|
||||
assertEquals("@Peer1 t2_syncup should be sync up and have 100 rows", 100,
|
||||
rowCount_ht2TargetAtPeer1);
|
||||
|
||||
for (int i = 0; i < NB_RETRIES; i++) {
|
||||
syncUp(utility1);
|
||||
rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
|
||||
rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
|
||||
if (i == NB_RETRIES - 1) {
|
||||
if (rowCount_ht1TargetAtPeer1 != 50 || rowCount_ht2TargetAtPeer1 != 100) {
|
||||
// syncUP still failed. Let's look at the source in case anything wrong there
|
||||
utility1.restartHBaseCluster(1);
|
||||
rowCount_ht1Source = utility1.countRows(ht1Source);
|
||||
LOG.debug("t1_syncup should have 51 rows at source, and it is " + rowCount_ht1Source);
|
||||
rowCount_ht2Source = utility1.countRows(ht2Source);
|
||||
LOG.debug("t2_syncup should have 101 rows at source, and it is " + rowCount_ht2Source);
|
||||
}
|
||||
assertEquals("@Peer1 t1_syncup should be sync up and have 50 rows", 50,
|
||||
rowCount_ht1TargetAtPeer1);
|
||||
assertEquals("@Peer1 t2_syncup should be sync up and have 100 rows", 100,
|
||||
rowCount_ht2TargetAtPeer1);
|
||||
}
|
||||
if (rowCount_ht1TargetAtPeer1 == 50 && rowCount_ht2TargetAtPeer1 == 100) {
|
||||
LOG.info("SyncUpAfterDelete succeeded at retry = " + i);
|
||||
break;
|
||||
} else {
|
||||
LOG.debug("SyncUpAfterDelete failed at retry = " + i + ", with rowCount_ht1TargetPeer1 ="
|
||||
+ rowCount_ht1TargetAtPeer1 + " and rowCount_ht2TargetAtPeer1 ="
|
||||
+ rowCount_ht2TargetAtPeer1);
|
||||
}
|
||||
Thread.sleep(SLEEP_TIME);
|
||||
}
|
||||
}
|
||||
|
||||
private void mimicSyncUpAfterPut() throws Exception {
|
||||
|
@ -350,14 +374,34 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
|
|||
rowCount_ht2TargetAtPeer1);
|
||||
|
||||
// after syun up
|
||||
syncUp(utility1);
|
||||
rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
|
||||
rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
|
||||
assertEquals("@Peer1 t1_syncup should be sync up and have 100 rows", 100,
|
||||
rowCount_ht1TargetAtPeer1);
|
||||
assertEquals("@Peer1 t2_syncup should be sync up and have 200 rows", 200,
|
||||
rowCount_ht2TargetAtPeer1);
|
||||
|
||||
for (int i = 0; i < NB_RETRIES; i++) {
|
||||
syncUp(utility1);
|
||||
rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
|
||||
rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
|
||||
if (i == NB_RETRIES - 1) {
|
||||
if (rowCount_ht1TargetAtPeer1 != 100 || rowCount_ht2TargetAtPeer1 != 200) {
|
||||
// syncUP still failed. Let's look at the source in case anything wrong there
|
||||
utility1.restartHBaseCluster(1);
|
||||
rowCount_ht1Source = utility1.countRows(ht1Source);
|
||||
LOG.debug("t1_syncup should have 102 rows at source, and it is " + rowCount_ht1Source);
|
||||
rowCount_ht2Source = utility1.countRows(ht2Source);
|
||||
LOG.debug("t2_syncup should have 202 rows at source, and it is " + rowCount_ht2Source);
|
||||
}
|
||||
assertEquals("@Peer1 t1_syncup should be sync up and have 100 rows", 100,
|
||||
rowCount_ht1TargetAtPeer1);
|
||||
assertEquals("@Peer1 t2_syncup should be sync up and have 200 rows", 200,
|
||||
rowCount_ht2TargetAtPeer1);
|
||||
}
|
||||
if (rowCount_ht1TargetAtPeer1 == 100 && rowCount_ht2TargetAtPeer1 == 200) {
|
||||
LOG.info("SyncUpAfterPut succeeded at retry = " + i);
|
||||
break;
|
||||
} else {
|
||||
LOG.debug("SyncUpAfterPut failed at retry = " + i + ", with rowCount_ht1TargetPeer1 ="
|
||||
+ rowCount_ht1TargetAtPeer1 + " and rowCount_ht2TargetAtPeer1 ="
|
||||
+ rowCount_ht2TargetAtPeer1);
|
||||
}
|
||||
Thread.sleep(SLEEP_TIME);
|
||||
}
|
||||
}
|
||||
|
||||
private void syncUp(HBaseTestingUtility ut) throws Exception {
|
||||
|
|
Loading…
Reference in New Issue