1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.replication;
19
20 import static org.junit.Assert.assertArrayEquals;
21
22 import java.util.concurrent.BlockingQueue;
23 import java.util.concurrent.LinkedBlockingQueue;
24 import java.util.concurrent.TimeUnit;
25 import java.util.concurrent.TimeoutException;
26 import org.apache.hadoop.hbase.Cell;
27 import org.apache.hadoop.hbase.CellUtil;
28 import org.apache.hadoop.hbase.HBaseTestingUtility;
29 import org.apache.hadoop.hbase.TableName;
30 import org.apache.hadoop.hbase.client.Put;
31 import org.apache.hadoop.hbase.client.Table;
32 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
33 import org.apache.hadoop.hbase.testclassification.MediumTests;
34 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
35 import org.apache.hadoop.hbase.util.Bytes;
36 import org.apache.hadoop.hbase.wal.WAL;
37 import org.junit.AfterClass;
38 import org.junit.BeforeClass;
39 import org.junit.Test;
40 import org.junit.experimental.categories.Category;
41 import org.slf4j.Logger;
42 import org.slf4j.LoggerFactory;
43
44
45
46
47 @Category({ ReplicationTests.class, MediumTests.class })
48 public class TestVerifyCellsReplicationEndpoint {
49
50 private static final Logger LOG =
51 LoggerFactory.getLogger(TestVerifyCellsReplicationEndpoint.class);
52
53 private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
54
55 private static final TableName TABLE_NAME = TableName.valueOf("empty");
56
57 private static final byte[] CF = Bytes.toBytes("family");
58
59 private static final byte[] CQ = Bytes.toBytes("qualifier");
60
61 private static final String PEER_ID = "empty";
62
63 private static final BlockingQueue<Cell> CELLS = new LinkedBlockingQueue<>();
64
65 public static final class EndpointForTest extends VerifyWALEntriesReplicationEndpoint {
66
67 @Override
68 public boolean replicate(ReplicateContext replicateContext) {
69 LOG.info(replicateContext.getEntries().toString());
70 for (WAL.Entry entry: replicateContext.getEntries()) {
71 CELLS.addAll(entry.getEdit().getCells());
72 }
73 return super.replicate(replicateContext);
74 }
75 }
76
77 @BeforeClass
78 public static void setUp() throws Exception {
79 UTIL.startMiniCluster(3);
80
81
82 ReplicationPeerConfig peerConfig = new ReplicationPeerConfig();
83 peerConfig.setClusterKey("zk1:8888:/hbase");
84 peerConfig.setReplicationEndpointImpl(EndpointForTest.class.getName());
85 UTIL.createTable(TABLE_NAME, CF);
86 try (ReplicationAdmin replAdmin = new ReplicationAdmin(UTIL.getConfiguration())) {
87 replAdmin.addPeer(PEER_ID, peerConfig);
88 }
89 }
90
91 @AfterClass
92 public static void tearDown() throws Exception {
93 UTIL.shutdownMiniCluster();
94 }
95
96 @Test
97 public void test() throws Exception {
98 try (Table table = UTIL.getConnection().getTable(TABLE_NAME)) {
99 for (int i = 0; i < 100; i++) {
100 table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
101 }
102 }
103 long lastNoCellTime = -1;
104 for (int i = 0; i < 100;) {
105 Cell cell = CELLS.poll();
106 if (cell == null) {
107 if (lastNoCellTime < 0) {
108 lastNoCellTime = System.nanoTime();
109 } else {
110 if (System.nanoTime() - lastNoCellTime >= TimeUnit.SECONDS.toNanos(30)) {
111 throw new TimeoutException("Timeout waiting for wal edit");
112 }
113 }
114 Thread.sleep(1000);
115 continue;
116 }
117 lastNoCellTime = -1;
118 if (!Bytes.equals(CF, CellUtil.cloneFamily(cell))) {
119
120 continue;
121 }
122 assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneRow(cell));
123 assertArrayEquals(CQ, CellUtil.cloneQualifier(cell));
124 assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneValue(cell));
125 i++;
126 }
127 }
128 }