View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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.Admin;
29  import org.apache.hadoop.hbase.client.Delete;
30  import org.apache.hadoop.hbase.client.HBaseAdmin;
31  import org.apache.hadoop.hbase.client.HTable;
32  import org.apache.hadoop.hbase.client.Put;
33  import org.apache.hadoop.hbase.client.Table;
34  import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
35  import org.apache.hadoop.hbase.testclassification.LargeTests;
36  import org.apache.hadoop.hbase.util.Bytes;
37  import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp;
38  import org.junit.Before;
39  import org.junit.Test;
40  import org.junit.experimental.categories.Category;
41  
42  @Category(LargeTests.class)
43  public class TestReplicationSyncUpTool extends TestReplicationBase {
44  
45    private static final Log LOG = LogFactory.getLog(TestReplicationSyncUpTool.class);
46  
47    private static final TableName t1_su = TableName.valueOf("t1_syncup");
48    private static final TableName t2_su = TableName.valueOf("t2_syncup");
49  
50    protected static final byte[] famName = Bytes.toBytes("cf1");
51    private static final byte[] qualName = Bytes.toBytes("q1");
52  
53    protected static final byte[] noRepfamName = Bytes.toBytes("norep");
54  
55    private HTableDescriptor t1_syncupSource, t1_syncupTarget;
56    private HTableDescriptor t2_syncupSource, t2_syncupTarget;
57  
58    protected Table ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1;
59  
60    @Before
61    public void setUp() throws Exception {
62  
63      HColumnDescriptor fam;
64  
65      t1_syncupSource = new HTableDescriptor(t1_su);
66      fam = new HColumnDescriptor(famName);
67      fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
68      t1_syncupSource.addFamily(fam);
69      fam = new HColumnDescriptor(noRepfamName);
70      t1_syncupSource.addFamily(fam);
71  
72      t1_syncupTarget = new HTableDescriptor(t1_su);
73      fam = new HColumnDescriptor(famName);
74      t1_syncupTarget.addFamily(fam);
75      fam = new HColumnDescriptor(noRepfamName);
76      t1_syncupTarget.addFamily(fam);
77  
78      t2_syncupSource = new HTableDescriptor(t2_su);
79      fam = new HColumnDescriptor(famName);
80      fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
81      t2_syncupSource.addFamily(fam);
82      fam = new HColumnDescriptor(noRepfamName);
83      t2_syncupSource.addFamily(fam);
84  
85      t2_syncupTarget = new HTableDescriptor(t2_su);
86      fam = new HColumnDescriptor(famName);
87      t2_syncupTarget.addFamily(fam);
88      fam = new HColumnDescriptor(noRepfamName);
89      t2_syncupTarget.addFamily(fam);
90  
91    }
92  
93    /**
94     * Add a row to a table in each cluster, check it's replicated, delete it,
95     * check's gone Also check the puts and deletes are not replicated back to
96     * the originating cluster.
97     */
98    @Test(timeout = 300000)
99    public void testSyncUpTool() throws Exception {
100 
101     /**
102      * Set up Replication: on Master and one Slave
103      * Table: t1_syncup and t2_syncup
104      * columnfamily:
105      *    'cf1'  : replicated
106      *    'norep': not replicated
107      */
108     setupReplication();
109 
110     /**
111      * at Master:
112      * t1_syncup: put 100 rows into cf1, and 1 rows into norep
113      * t2_syncup: put 200 rows into cf1, and 1 rows into norep
114      *
115      * verify correctly replicated to slave
116      */
117     putAndReplicateRows();
118 
119     /**
120      * Verify delete works
121      *
122      * step 1: stop hbase on Slave
123      *
124      * step 2: at Master:
125      *  t1_syncup: delete 50 rows  from cf1
126      *  t2_syncup: delete 100 rows from cf1
127      *  no change on 'norep'
128      *
129      * step 3: stop hbase on master, restart hbase on Slave
130      *
131      * step 4: verify Slave still have the rows before delete
132      *      t1_syncup: 100 rows from cf1
133      *      t2_syncup: 200 rows from cf1
134      *
135      * step 5: run syncup tool on Master
136      *
137      * step 6: verify that delete show up on Slave
138      *      t1_syncup: 50 rows from cf1
139      *      t2_syncup: 100 rows from cf1
140      *
141      * verify correctly replicated to Slave
142      */
143     mimicSyncUpAfterDelete();
144 
145     /**
146      * Verify put works
147      *
148      * step 1: stop hbase on Slave
149      *
150      * step 2: at Master:
151      *  t1_syncup: put 100 rows  from cf1
152      *  t2_syncup: put 200 rows  from cf1
153      *  and put another row on 'norep'
154      *  ATTN: put to 'cf1' will overwrite existing rows, so end count will
155      *        be 100 and 200 respectively
156      *      put to 'norep' will add a new row.
157      *
158      * step 3: stop hbase on master, restart hbase on Slave
159      *
160      * step 4: verify Slave still has the rows before put
161      *      t1_syncup: 50 rows from cf1
162      *      t2_syncup: 100 rows from cf1
163      *
164      * step 5: run syncup tool on Master
165      *
166      * step 6: verify that put show up on Slave
167      *         and 'norep' does not
168      *      t1_syncup: 100 rows from cf1
169      *      t2_syncup: 200 rows from cf1
170      *
171      * verify correctly replicated to Slave
172      */
173     mimicSyncUpAfterPut();
174 
175   }
176 
177   protected void setupReplication() throws Exception {
178     ReplicationAdmin admin1 = new ReplicationAdmin(conf1);
179     ReplicationAdmin admin2 = new ReplicationAdmin(conf2);
180 
181     Admin ha = new HBaseAdmin(conf1);
182     ha.createTable(t1_syncupSource);
183     ha.createTable(t2_syncupSource);
184     ha.close();
185 
186     ha = new HBaseAdmin(conf2);
187     ha.createTable(t1_syncupTarget);
188     ha.createTable(t2_syncupTarget);
189     ha.close();
190 
191     // Get HTable from Master
192     ht1Source = new HTable(conf1, t1_su);
193     ht1Source.setWriteBufferSize(1024);
194     ht2Source = new HTable(conf1, t2_su);
195     ht1Source.setWriteBufferSize(1024);
196 
197     // Get HTable from Peer1
198     ht1TargetAtPeer1 = new HTable(conf2, t1_su);
199     ht1TargetAtPeer1.setWriteBufferSize(1024);
200     ht2TargetAtPeer1 = new HTable(conf2, t2_su);
201     ht2TargetAtPeer1.setWriteBufferSize(1024);
202 
203     /**
204      * set M-S : Master: utility1 Slave1: utility2
205      */
206     ReplicationPeerConfig rpc = new ReplicationPeerConfig();
207     rpc.setClusterKey(utility2.getClusterKey());
208     admin1.addPeer("1", rpc);
209 
210     admin1.close();
211     admin2.close();
212   }
213 
214   private void putAndReplicateRows() throws Exception {
215     LOG.debug("putAndReplicateRows");
216     // add rows to Master cluster,
217     Put p;
218 
219     // 100 + 1 row to t1_syncup
220     for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
221       p = new Put(Bytes.toBytes("row" + i));
222       p.add(famName, qualName, Bytes.toBytes("val" + i));
223       ht1Source.put(p);
224     }
225     p = new Put(Bytes.toBytes("row" + 9999));
226     p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9999));
227     ht1Source.put(p);
228 
229     // 200 + 1 row to t2_syncup
230     for (int i = 0; i < NB_ROWS_IN_BATCH * 2; i++) {
231       p = new Put(Bytes.toBytes("row" + i));
232       p.add(famName, qualName, Bytes.toBytes("val" + i));
233       ht2Source.put(p);
234     }
235     p = new Put(Bytes.toBytes("row" + 9999));
236     p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9999));
237     ht2Source.put(p);
238 
239     // ensure replication completed
240     Thread.sleep(SLEEP_TIME);
241     int rowCount_ht1Source = utility1.countRows(ht1Source);
242     for (int i = 0; i < NB_RETRIES; i++) {
243       int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
244       if (i==NB_RETRIES-1) {
245         assertEquals("t1_syncup has 101 rows on source, and 100 on slave1", rowCount_ht1Source - 1,
246             rowCount_ht1TargetAtPeer1);
247       }
248       if (rowCount_ht1Source - 1 == rowCount_ht1TargetAtPeer1) {
249         break;
250       }
251       Thread.sleep(SLEEP_TIME);
252     }
253 
254     int rowCount_ht2Source = utility1.countRows(ht2Source);
255     for (int i = 0; i < NB_RETRIES; i++) {
256       int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
257       if (i==NB_RETRIES-1) {
258         assertEquals("t2_syncup has 201 rows on source, and 200 on slave1", rowCount_ht2Source - 1,
259             rowCount_ht2TargetAtPeer1);
260       }
261       if (rowCount_ht2Source - 1 == rowCount_ht2TargetAtPeer1) {
262         break;
263       }
264       Thread.sleep(SLEEP_TIME);
265     }
266   }
267 
268   private void mimicSyncUpAfterDelete() throws Exception {
269     LOG.debug("mimicSyncUpAfterDelete");
270     utility2.shutdownMiniHBaseCluster();
271 
272     List<Delete> list = new ArrayList<Delete>();
273     // delete half of the rows
274     for (int i = 0; i < NB_ROWS_IN_BATCH / 2; i++) {
275       String rowKey = "row" + i;
276       Delete del = new Delete(rowKey.getBytes());
277       list.add(del);
278     }
279     ht1Source.delete(list);
280 
281     for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
282       String rowKey = "row" + i;
283       Delete del = new Delete(rowKey.getBytes());
284       list.add(del);
285     }
286     ht2Source.delete(list);
287 
288     int rowCount_ht1Source = utility1.countRows(ht1Source);
289     assertEquals("t1_syncup has 51 rows on source, after remove 50 of the replicated colfam", 51,
290       rowCount_ht1Source);
291 
292     int rowCount_ht2Source = utility1.countRows(ht2Source);
293     assertEquals("t2_syncup has 101 rows on source, after remove 100 of the replicated colfam",
294       101, rowCount_ht2Source);
295 
296     utility1.shutdownMiniHBaseCluster();
297     utility2.restartHBaseCluster(1);
298 
299     Thread.sleep(SLEEP_TIME);
300 
301     // before sync up
302     int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
303     int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
304     assertEquals("@Peer1 t1_syncup should still have 100 rows", 100, rowCount_ht1TargetAtPeer1);
305     assertEquals("@Peer1 t2_syncup should still have 200 rows", 200, rowCount_ht2TargetAtPeer1);
306 
307     // After sync up
308     for (int i = 0; i < NB_RETRIES; i++) {
309       syncUp(utility1);
310       rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
311       rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
312       if (i == NB_RETRIES - 1) {
313         if (rowCount_ht1TargetAtPeer1 != 50 || rowCount_ht2TargetAtPeer1 != 100) {
314           // syncUP still failed. Let's look at the source in case anything wrong there
315           utility1.restartHBaseCluster(1);
316           rowCount_ht1Source = utility1.countRows(ht1Source);
317           LOG.debug("t1_syncup should have 51 rows at source, and it is " + rowCount_ht1Source);
318           rowCount_ht2Source = utility1.countRows(ht2Source);
319           LOG.debug("t2_syncup should have 101 rows at source, and it is " + rowCount_ht2Source);
320         }
321         assertEquals("@Peer1 t1_syncup should be sync up and have 50 rows", 50,
322           rowCount_ht1TargetAtPeer1);
323         assertEquals("@Peer1 t2_syncup should be sync up and have 100 rows", 100,
324           rowCount_ht2TargetAtPeer1);
325       }
326       if (rowCount_ht1TargetAtPeer1 == 50 && rowCount_ht2TargetAtPeer1 == 100) {
327         LOG.info("SyncUpAfterDelete succeeded at retry = " + i);
328         break;
329       } else {
330         LOG.debug("SyncUpAfterDelete failed at retry = " + i + ", with rowCount_ht1TargetPeer1 ="
331             + rowCount_ht1TargetAtPeer1 + " and rowCount_ht2TargetAtPeer1 ="
332             + rowCount_ht2TargetAtPeer1);
333       }
334       Thread.sleep(SLEEP_TIME);
335     }
336   }
337 
338   private void mimicSyncUpAfterPut() throws Exception {
339     LOG.debug("mimicSyncUpAfterPut");
340     utility1.restartHBaseCluster(1);
341     utility2.shutdownMiniHBaseCluster();
342 
343     Put p;
344     // another 100 + 1 row to t1_syncup
345     // we should see 100 + 2 rows now
346     for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
347       p = new Put(Bytes.toBytes("row" + i));
348       p.add(famName, qualName, Bytes.toBytes("val" + i));
349       ht1Source.put(p);
350     }
351     p = new Put(Bytes.toBytes("row" + 9998));
352     p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9998));
353     ht1Source.put(p);
354 
355     // another 200 + 1 row to t1_syncup
356     // we should see 200 + 2 rows now
357     for (int i = 0; i < NB_ROWS_IN_BATCH * 2; i++) {
358       p = new Put(Bytes.toBytes("row" + i));
359       p.add(famName, qualName, Bytes.toBytes("val" + i));
360       ht2Source.put(p);
361     }
362     p = new Put(Bytes.toBytes("row" + 9998));
363     p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9998));
364     ht2Source.put(p);
365 
366     int rowCount_ht1Source = utility1.countRows(ht1Source);
367     assertEquals("t1_syncup has 102 rows on source", 102, rowCount_ht1Source);
368     int rowCount_ht2Source = utility1.countRows(ht2Source);
369     assertEquals("t2_syncup has 202 rows on source", 202, rowCount_ht2Source);
370 
371     utility1.shutdownMiniHBaseCluster();
372     utility2.restartHBaseCluster(1);
373 
374     Thread.sleep(SLEEP_TIME);
375 
376     // before sync up
377     int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
378     int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
379     assertEquals("@Peer1 t1_syncup should be NOT sync up and have 50 rows", 50,
380       rowCount_ht1TargetAtPeer1);
381     assertEquals("@Peer1 t2_syncup should be NOT sync up and have 100 rows", 100,
382       rowCount_ht2TargetAtPeer1);
383 
384     // after syun up
385     for (int i = 0; i < NB_RETRIES; i++) {
386       syncUp(utility1);
387       rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
388       rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
389       if (i == NB_RETRIES - 1) {
390         if (rowCount_ht1TargetAtPeer1 != 100 || rowCount_ht2TargetAtPeer1 != 200) {
391           // syncUP still failed. Let's look at the source in case anything wrong there
392           utility1.restartHBaseCluster(1);
393           rowCount_ht1Source = utility1.countRows(ht1Source);
394           LOG.debug("t1_syncup should have 102 rows at source, and it is " + rowCount_ht1Source);
395           rowCount_ht2Source = utility1.countRows(ht2Source);
396           LOG.debug("t2_syncup should have 202 rows at source, and it is " + rowCount_ht2Source);
397         }
398         assertEquals("@Peer1 t1_syncup should be sync up and have 100 rows", 100,
399           rowCount_ht1TargetAtPeer1);
400         assertEquals("@Peer1 t2_syncup should be sync up and have 200 rows", 200,
401           rowCount_ht2TargetAtPeer1);
402       }
403       if (rowCount_ht1TargetAtPeer1 == 100 && rowCount_ht2TargetAtPeer1 == 200) {
404         LOG.info("SyncUpAfterPut succeeded at retry = " + i);
405         break;
406       } else {
407         LOG.debug("SyncUpAfterPut failed at retry = " + i + ", with rowCount_ht1TargetPeer1 ="
408             + rowCount_ht1TargetAtPeer1 + " and rowCount_ht2TargetAtPeer1 ="
409             + rowCount_ht2TargetAtPeer1);
410       }
411       Thread.sleep(SLEEP_TIME);
412     }
413   }
414 
415   protected void syncUp(HBaseTestingUtility ut) throws Exception {
416     ReplicationSyncUp.setConfigure(ut.getConfiguration());
417     String[] arguments = new String[] { null };
418     new ReplicationSyncUp().run(arguments);
419   }
420 
421 }