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.assertEquals;
21
22 import java.util.ArrayList;
23 import java.util.List;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.hbase.*;
28 import org.apache.hadoop.hbase.client.Delete;
29 import org.apache.hadoop.hbase.client.HBaseAdmin;
30 import org.apache.hadoop.hbase.client.HTable;
31 import org.apache.hadoop.hbase.client.Put;
32 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
33 import org.apache.hadoop.hbase.util.Bytes;
34 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp;
35 import org.junit.Before;
36 import org.junit.Test;
37 import org.junit.experimental.categories.Category;
38
39 @Category(LargeTests.class)
40 public class TestReplicationSyncUpTool extends TestReplicationBase {
41
42 private static final Log LOG = LogFactory.getLog(TestReplicationSyncUpTool.class);
43
44 private static final byte[] t1_su = Bytes.toBytes("t1_syncup");
45 private static final byte[] t2_su = Bytes.toBytes("t2_syncup");
46
47 private static final byte[] famName = Bytes.toBytes("cf1");
48 private static final byte[] qualName = Bytes.toBytes("q1");
49
50 private static final byte[] noRepfamName = Bytes.toBytes("norep");
51
52 private HTableDescriptor t1_syncupSource, t1_syncupTarget;
53 private HTableDescriptor t2_syncupSource, t2_syncupTarget;
54
55 private HTable ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1;
56
57 @Before
58 public void setUp() throws Exception {
59
60 HColumnDescriptor fam;
61
62 t1_syncupSource = new HTableDescriptor(TableName.valueOf(t1_su));
63 fam = new HColumnDescriptor(famName);
64 fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
65 t1_syncupSource.addFamily(fam);
66 fam = new HColumnDescriptor(noRepfamName);
67 t1_syncupSource.addFamily(fam);
68
69 t1_syncupTarget = new HTableDescriptor(TableName.valueOf(t1_su));
70 fam = new HColumnDescriptor(famName);
71 t1_syncupTarget.addFamily(fam);
72 fam = new HColumnDescriptor(noRepfamName);
73 t1_syncupTarget.addFamily(fam);
74
75 t2_syncupSource = new HTableDescriptor(TableName.valueOf(t2_su));
76 fam = new HColumnDescriptor(famName);
77 fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
78 t2_syncupSource.addFamily(fam);
79 fam = new HColumnDescriptor(noRepfamName);
80 t2_syncupSource.addFamily(fam);
81
82 t2_syncupTarget = new HTableDescriptor(TableName.valueOf(t2_su));
83 fam = new HColumnDescriptor(famName);
84 t2_syncupTarget.addFamily(fam);
85 fam = new HColumnDescriptor(noRepfamName);
86 t2_syncupTarget.addFamily(fam);
87
88 }
89
90
91
92
93
94
95 @Test(timeout = 300000)
96 public void testSyncUpTool() throws Exception {
97
98
99
100
101
102
103
104
105 setupReplication();
106
107
108
109
110
111
112
113
114 putAndReplicateRows();
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140 mimicSyncUpAfterDelete();
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170 mimicSyncUpAfterPut();
171
172 }
173
174 private void setupReplication() throws Exception {
175 ReplicationAdmin admin1 = new ReplicationAdmin(conf1);
176 ReplicationAdmin admin2 = new ReplicationAdmin(conf2);
177
178 HBaseAdmin ha = new HBaseAdmin(conf1);
179 ha.createTable(t1_syncupSource);
180 ha.createTable(t2_syncupSource);
181 ha.close();
182
183 ha = new HBaseAdmin(conf2);
184 ha.createTable(t1_syncupTarget);
185 ha.createTable(t2_syncupTarget);
186 ha.close();
187
188
189 ht1Source = new HTable(conf1, t1_su);
190 ht1Source.setWriteBufferSize(1024);
191 ht2Source = new HTable(conf1, t2_su);
192 ht1Source.setWriteBufferSize(1024);
193
194
195 ht1TargetAtPeer1 = new HTable(conf2, t1_su);
196 ht1TargetAtPeer1.setWriteBufferSize(1024);
197 ht2TargetAtPeer1 = new HTable(conf2, t2_su);
198 ht2TargetAtPeer1.setWriteBufferSize(1024);
199
200
201
202
203 admin1.addPeer("1", utility2.getClusterKey());
204
205 admin1.close();
206 admin2.close();
207 }
208
209 private void putAndReplicateRows() throws Exception {
210
211 Put p;
212
213
214 for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
215 p = new Put(Bytes.toBytes("row" + i));
216 p.add(famName, qualName, Bytes.toBytes("val" + i));
217 ht1Source.put(p);
218 }
219 p = new Put(Bytes.toBytes("row" + 9999));
220 p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9999));
221 ht1Source.put(p);
222
223
224 for (int i = 0; i < NB_ROWS_IN_BATCH * 2; i++) {
225 p = new Put(Bytes.toBytes("row" + i));
226 p.add(famName, qualName, Bytes.toBytes("val" + i));
227 ht2Source.put(p);
228 }
229 p = new Put(Bytes.toBytes("row" + 9999));
230 p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9999));
231 ht2Source.put(p);
232
233
234 Thread.sleep(SLEEP_TIME);
235 int rowCount_ht1Source = utility1.countRows(ht1Source);
236 for (int i = 0; i < NB_RETRIES; i++) {
237 int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
238 if (i==NB_RETRIES-1) {
239 assertEquals("t1_syncup has 101 rows on source, and 100 on slave1", rowCount_ht1Source - 1,
240 rowCount_ht1TargetAtPeer1);
241 }
242 if (rowCount_ht1Source - 1 == rowCount_ht1TargetAtPeer1) {
243 break;
244 }
245 Thread.sleep(SLEEP_TIME);
246 }
247
248 int rowCount_ht2Source = utility1.countRows(ht2Source);
249 for (int i = 0; i < NB_RETRIES; i++) {
250 int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
251 if (i==NB_RETRIES-1) {
252 assertEquals("t2_syncup has 201 rows on source, and 200 on slave1", rowCount_ht2Source - 1,
253 rowCount_ht2TargetAtPeer1);
254 }
255 if (rowCount_ht2Source - 1 == rowCount_ht2TargetAtPeer1) {
256 break;
257 }
258 Thread.sleep(SLEEP_TIME);
259 }
260 }
261
262 private void mimicSyncUpAfterDelete() throws Exception {
263 utility2.shutdownMiniHBaseCluster();
264
265 List<Delete> list = new ArrayList<Delete>();
266
267 for (int i = 0; i < NB_ROWS_IN_BATCH / 2; i++) {
268 String rowKey = "row" + i;
269 Delete del = new Delete(rowKey.getBytes());
270 list.add(del);
271 }
272 ht1Source.delete(list);
273
274 for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
275 String rowKey = "row" + i;
276 Delete del = new Delete(rowKey.getBytes());
277 list.add(del);
278 }
279 ht2Source.delete(list);
280
281 int rowCount_ht1Source = utility1.countRows(ht1Source);
282 assertEquals("t1_syncup has 51 rows on source, after remove 50 of the replicated colfam", 51,
283 rowCount_ht1Source);
284
285 int rowCount_ht2Source = utility1.countRows(ht2Source);
286 assertEquals("t2_syncup has 101 rows on source, after remove 100 of the replicated colfam",
287 101, rowCount_ht2Source);
288
289 utility1.shutdownMiniHBaseCluster();
290 utility2.restartHBaseCluster(1);
291
292 Thread.sleep(SLEEP_TIME);
293
294
295 int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
296 int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
297 assertEquals("@Peer1 t1_syncup should still have 100 rows", 100, rowCount_ht1TargetAtPeer1);
298 assertEquals("@Peer1 t2_syncup should still have 200 rows", 200, rowCount_ht2TargetAtPeer1);
299
300
301 for (int i = 0; i < NB_RETRIES; i++) {
302 syncUp(utility1);
303 rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
304 rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
305 if (i == NB_RETRIES - 1) {
306 if (rowCount_ht1TargetAtPeer1 != 50 || rowCount_ht2TargetAtPeer1 != 100) {
307
308 utility1.restartHBaseCluster(1);
309 rowCount_ht1Source = utility1.countRows(ht1Source);
310 LOG.debug("t1_syncup should have 51 rows at source, and it is " + rowCount_ht1Source);
311 rowCount_ht2Source = utility1.countRows(ht2Source);
312 LOG.debug("t2_syncup should have 101 rows at source, and it is " + rowCount_ht2Source);
313 }
314 assertEquals("@Peer1 t1_syncup should be sync up and have 50 rows", 50,
315 rowCount_ht1TargetAtPeer1);
316 assertEquals("@Peer1 t2_syncup should be sync up and have 100 rows", 100,
317 rowCount_ht2TargetAtPeer1);
318 }
319 if (rowCount_ht1TargetAtPeer1 == 50 && rowCount_ht2TargetAtPeer1 == 100) {
320 LOG.info("SyncUpAfterDelete succeeded at retry = " + i);
321 break;
322 } else {
323 LOG.debug("SyncUpAfterDelete failed at retry = " + i + ", with rowCount_ht1TargetPeer1 ="
324 + rowCount_ht1TargetAtPeer1 + " and rowCount_ht2TargetAtPeer1 ="
325 + rowCount_ht2TargetAtPeer1);
326 }
327 Thread.sleep(SLEEP_TIME);
328 }
329 }
330
331 private void mimicSyncUpAfterPut() throws Exception {
332 utility1.restartHBaseCluster(1);
333 utility2.shutdownMiniHBaseCluster();
334
335 Put p;
336
337
338 for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
339 p = new Put(Bytes.toBytes("row" + i));
340 p.add(famName, qualName, Bytes.toBytes("val" + i));
341 ht1Source.put(p);
342 }
343 p = new Put(Bytes.toBytes("row" + 9998));
344 p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9998));
345 ht1Source.put(p);
346
347
348
349 for (int i = 0; i < NB_ROWS_IN_BATCH * 2; i++) {
350 p = new Put(Bytes.toBytes("row" + i));
351 p.add(famName, qualName, Bytes.toBytes("val" + i));
352 ht2Source.put(p);
353 }
354 p = new Put(Bytes.toBytes("row" + 9998));
355 p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9998));
356 ht2Source.put(p);
357
358 int rowCount_ht1Source = utility1.countRows(ht1Source);
359 assertEquals("t1_syncup has 102 rows on source", 102, rowCount_ht1Source);
360 int rowCount_ht2Source = utility1.countRows(ht2Source);
361 assertEquals("t2_syncup has 202 rows on source", 202, rowCount_ht2Source);
362
363 utility1.shutdownMiniHBaseCluster();
364 utility2.restartHBaseCluster(1);
365
366 Thread.sleep(SLEEP_TIME);
367
368
369 int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
370 int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
371 assertEquals("@Peer1 t1_syncup should be NOT sync up and have 50 rows", 50,
372 rowCount_ht1TargetAtPeer1);
373 assertEquals("@Peer1 t2_syncup should be NOT sync up and have 100 rows", 100,
374 rowCount_ht2TargetAtPeer1);
375
376
377 for (int i = 0; i < NB_RETRIES; i++) {
378 syncUp(utility1);
379 rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
380 rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
381 if (i == NB_RETRIES - 1) {
382 if (rowCount_ht1TargetAtPeer1 != 100 || rowCount_ht2TargetAtPeer1 != 200) {
383
384 utility1.restartHBaseCluster(1);
385 rowCount_ht1Source = utility1.countRows(ht1Source);
386 LOG.debug("t1_syncup should have 102 rows at source, and it is " + rowCount_ht1Source);
387 rowCount_ht2Source = utility1.countRows(ht2Source);
388 LOG.debug("t2_syncup should have 202 rows at source, and it is " + rowCount_ht2Source);
389 }
390 assertEquals("@Peer1 t1_syncup should be sync up and have 100 rows", 100,
391 rowCount_ht1TargetAtPeer1);
392 assertEquals("@Peer1 t2_syncup should be sync up and have 200 rows", 200,
393 rowCount_ht2TargetAtPeer1);
394 }
395 if (rowCount_ht1TargetAtPeer1 == 100 && rowCount_ht2TargetAtPeer1 == 200) {
396 LOG.info("SyncUpAfterPut succeeded at retry = " + i);
397 break;
398 } else {
399 LOG.debug("SyncUpAfterPut failed at retry = " + i + ", with rowCount_ht1TargetPeer1 ="
400 + rowCount_ht1TargetAtPeer1 + " and rowCount_ht2TargetAtPeer1 ="
401 + rowCount_ht2TargetAtPeer1);
402 }
403 Thread.sleep(SLEEP_TIME);
404 }
405 }
406
407 private void syncUp(HBaseTestingUtility ut) throws Exception {
408 ReplicationSyncUp.setConfigure(ut.getConfiguration());
409 String[] arguments = new String[] { null };
410 new ReplicationSyncUp().run(arguments);
411 }
412
413 }