HBASE-10189 Intermittent TestReplicationSyncUpTool failure (LarsH and Demai Ni)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1551788 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
larsh 2013-12-18 00:55:31 +00:00
parent 5341bb6f1a
commit 3e84d6e6b1
1 changed files with 32 additions and 24 deletions

View File

@ -22,8 +22,6 @@ import static org.junit.Assert.assertEquals;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; 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.*;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
@ -39,8 +37,6 @@ import org.junit.experimental.categories.Category;
@Category(LargeTests.class) @Category(LargeTests.class)
public class TestReplicationSyncUpTool extends TestReplicationBase { 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[] t1_su = Bytes.toBytes("t1_syncup");
private static final byte[] t2_su = Bytes.toBytes("t2_syncup"); private static final byte[] t2_su = Bytes.toBytes("t2_syncup");
@ -53,8 +49,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
private HTableDescriptor t2_syncupSource, t2_syncupTarget; private HTableDescriptor t2_syncupSource, t2_syncupTarget;
private HTable ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1; private HTable ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1;
private int rowCount_ht1Source, rowCount_ht2Source, rowCount_ht1TargetAtPeer1,
rowCount_ht2TargetAtPeer1;
@Before @Before
public void setUp() throws Exception { public void setUp() throws Exception {
@ -234,17 +228,31 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
// ensure replication completed // ensure replication completed
Thread.sleep(SLEEP_TIME); Thread.sleep(SLEEP_TIME);
int rowCount_ht1Source = utility1.countRows(ht1Source);
rowCount_ht1Source = utility1.countRows(ht1Source); for (int i = 0; i < NB_RETRIES; i++) {
rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1); int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
if (i==NB_RETRIES-1) {
assertEquals("t1_syncup has 101 rows on source, and 100 on slave1", rowCount_ht1Source - 1, assertEquals("t1_syncup has 101 rows on source, and 100 on slave1", rowCount_ht1Source - 1,
rowCount_ht1TargetAtPeer1); rowCount_ht1TargetAtPeer1);
}
if (rowCount_ht1Source - 1 == rowCount_ht1TargetAtPeer1) {
break;
}
Thread.sleep(SLEEP_TIME);
}
rowCount_ht2Source = utility1.countRows(ht2Source); int rowCount_ht2Source = utility1.countRows(ht2Source);
rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1); for (int i = 0; i < NB_RETRIES; i++) {
int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
if (i==NB_RETRIES-1) {
assertEquals("t2_syncup has 201 rows on source, and 200 on slave1", rowCount_ht2Source - 1, assertEquals("t2_syncup has 201 rows on source, and 200 on slave1", rowCount_ht2Source - 1,
rowCount_ht2TargetAtPeer1); rowCount_ht2TargetAtPeer1);
}
if (rowCount_ht2Source - 1 == rowCount_ht2TargetAtPeer1) {
break;
}
Thread.sleep(SLEEP_TIME);
}
} }
private void mimicSyncUpAfterDelete() throws Exception { private void mimicSyncUpAfterDelete() throws Exception {
@ -266,11 +274,11 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
} }
ht2Source.delete(list); ht2Source.delete(list);
rowCount_ht1Source = utility1.countRows(ht1Source); int rowCount_ht1Source = utility1.countRows(ht1Source);
assertEquals("t1_syncup has 51 rows on source, after remove 50 of the replicated colfam", 51, assertEquals("t1_syncup has 51 rows on source, after remove 50 of the replicated colfam", 51,
rowCount_ht1Source); rowCount_ht1Source);
rowCount_ht2Source = utility1.countRows(ht2Source); int rowCount_ht2Source = utility1.countRows(ht2Source);
assertEquals("t2_syncup has 101 rows on source, after remove 100 of the replicated colfam", assertEquals("t2_syncup has 101 rows on source, after remove 100 of the replicated colfam",
101, rowCount_ht2Source); 101, rowCount_ht2Source);
@ -280,8 +288,8 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
Thread.sleep(SLEEP_TIME); Thread.sleep(SLEEP_TIME);
// before sync up // before sync up
rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1); int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1); int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
assertEquals("@Peer1 t1_syncup should still have 100 rows", 100, rowCount_ht1TargetAtPeer1); assertEquals("@Peer1 t1_syncup should still have 100 rows", 100, rowCount_ht1TargetAtPeer1);
assertEquals("@Peer1 t2_syncup should still have 200 rows", 200, rowCount_ht2TargetAtPeer1); assertEquals("@Peer1 t2_syncup should still have 200 rows", 200, rowCount_ht2TargetAtPeer1);
@ -323,9 +331,9 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9998)); p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9998));
ht2Source.put(p); ht2Source.put(p);
rowCount_ht1Source = utility1.countRows(ht1Source); int rowCount_ht1Source = utility1.countRows(ht1Source);
assertEquals("t1_syncup has 102 rows on source", 102, rowCount_ht1Source); assertEquals("t1_syncup has 102 rows on source", 102, rowCount_ht1Source);
rowCount_ht2Source = utility1.countRows(ht2Source); int rowCount_ht2Source = utility1.countRows(ht2Source);
assertEquals("t2_syncup has 202 rows on source", 202, rowCount_ht2Source); assertEquals("t2_syncup has 202 rows on source", 202, rowCount_ht2Source);
utility1.shutdownMiniHBaseCluster(); utility1.shutdownMiniHBaseCluster();
@ -334,8 +342,8 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
Thread.sleep(SLEEP_TIME); Thread.sleep(SLEEP_TIME);
// before sync up // before sync up
rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1); int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1); int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
assertEquals("@Peer1 t1_syncup should be NOT sync up and have 50 rows", 50, assertEquals("@Peer1 t1_syncup should be NOT sync up and have 50 rows", 50,
rowCount_ht1TargetAtPeer1); rowCount_ht1TargetAtPeer1);
assertEquals("@Peer1 t2_syncup should be NOT sync up and have 100 rows", 100, assertEquals("@Peer1 t2_syncup should be NOT sync up and have 100 rows", 100,