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.hadoop.hbase.*;
26  import org.apache.hadoop.hbase.client.Delete;
27  import org.apache.hadoop.hbase.client.HBaseAdmin;
28  import org.apache.hadoop.hbase.client.HTable;
29  import org.apache.hadoop.hbase.client.Put;
30  import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
31  import org.apache.hadoop.hbase.util.Bytes;
32  import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp;
33  import org.junit.Before;
34  import org.junit.Test;
35  import org.junit.experimental.categories.Category;
36  
37  @Category(LargeTests.class)
38  public class TestReplicationSyncUpTool extends TestReplicationBase {
39  
40    private static final byte[] t1_su = Bytes.toBytes("t1_syncup");
41    private static final byte[] t2_su = Bytes.toBytes("t2_syncup");
42  
43    private static final byte[] famName = Bytes.toBytes("cf1");
44    private static final byte[] qualName = Bytes.toBytes("q1");
45  
46    private static final byte[] noRepfamName = Bytes.toBytes("norep");
47  
48    private HTableDescriptor t1_syncupSource, t1_syncupTarget;
49    private HTableDescriptor t2_syncupSource, t2_syncupTarget;
50  
51    private HTable ht1Source, ht2Source, ht1TargetAtPeer1, ht2TargetAtPeer1;
52  
53    @Before
54    public void setUp() throws Exception {
55  
56      HColumnDescriptor fam;
57  
58      t1_syncupSource = new HTableDescriptor(t1_su);
59      fam = new HColumnDescriptor(famName);
60      fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
61      t1_syncupSource.addFamily(fam);
62      fam = new HColumnDescriptor(noRepfamName);
63      t1_syncupSource.addFamily(fam);
64  
65      t1_syncupTarget = new HTableDescriptor(t1_su);
66      fam = new HColumnDescriptor(famName);
67      t1_syncupTarget.addFamily(fam);
68      fam = new HColumnDescriptor(noRepfamName);
69      t1_syncupTarget.addFamily(fam);
70  
71      t2_syncupSource = new HTableDescriptor(t2_su);
72      fam = new HColumnDescriptor(famName);
73      fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
74      t2_syncupSource.addFamily(fam);
75      fam = new HColumnDescriptor(noRepfamName);
76      t2_syncupSource.addFamily(fam);
77  
78      t2_syncupTarget = new HTableDescriptor(t2_su);
79      fam = new HColumnDescriptor(famName);
80      t2_syncupTarget.addFamily(fam);
81      fam = new HColumnDescriptor(noRepfamName);
82      t2_syncupTarget.addFamily(fam);
83  
84    }
85  
86    /**
87     * Add a row to a table in each cluster, check it's replicated, delete it,
88     * check's gone Also check the puts and deletes are not replicated back to
89     * the originating cluster.
90     */
91    @Test(timeout = 300000)
92    public void testSyncUpTool() throws Exception {
93  
94      /**
95       * Set up Replication: on Master and one Slave
96       * Table: t1_syncup and t2_syncup
97       * columnfamily:
98       *    'cf1'  : replicated
99       *    'norep': not replicated
100      */
101     setupReplication();
102 
103     /**
104      * at Master:
105      * t1_syncup: put 100 rows into cf1, and 1 rows into norep
106      * t2_syncup: put 200 rows into cf1, and 1 rows into norep
107      *
108      * verify correctly replicated to slave
109      */
110     putAndReplicateRows();
111 
112     /**
113      * Verify delete works
114      *
115      * step 1: stop hbase on Slave
116      *
117      * step 2: at Master:
118      *  t1_syncup: delete 50 rows  from cf1
119      *  t2_syncup: delete 100 rows from cf1
120      *  no change on 'norep'
121      *
122      * step 3: stop hbase on master, restart hbase on Slave
123      *
124      * step 4: verify Slave still have the rows before delete
125      *      t1_syncup: 100 rows from cf1
126      *      t2_syncup: 200 rows from cf1
127      *
128      * step 5: run syncup tool on Master
129      *
130      * step 6: verify that delete show up on Slave
131      *      t1_syncup: 50 rows from cf1
132      *      t2_syncup: 100 rows from cf1
133      *
134      * verify correctly replicated to Slave
135      */
136     mimicSyncUpAfterDelete();
137 
138     /**
139      * Verify put works
140      *
141      * step 1: stop hbase on Slave
142      *
143      * step 2: at Master:
144      *  t1_syncup: put 100 rows  from cf1
145      *  t2_syncup: put 200 rows  from cf1
146      *  and put another row on 'norep'
147      *  ATTN: put to 'cf1' will overwrite existing rows, so end count will
148      *        be 100 and 200 respectively
149      *      put to 'norep' will add a new row.
150      *
151      * step 3: stop hbase on master, restart hbase on Slave
152      *
153      * step 4: verify Slave still has the rows before put
154      *      t1_syncup: 50 rows from cf1
155      *      t2_syncup: 100 rows from cf1
156      *
157      * step 5: run syncup tool on Master
158      *
159      * step 6: verify that put show up on Slave
160      *         and 'norep' does not
161      *      t1_syncup: 100 rows from cf1
162      *      t2_syncup: 200 rows from cf1
163      *
164      * verify correctly replicated to Slave
165      */
166     mimicSyncUpAfterPut();
167 
168   }
169 
170   private void setupReplication() throws Exception {
171     ReplicationAdmin admin1 = new ReplicationAdmin(conf1);
172     ReplicationAdmin admin2 = new ReplicationAdmin(conf2);
173 
174     HBaseAdmin ha = new HBaseAdmin(conf1);
175     ha.createTable(t1_syncupSource);
176     ha.createTable(t2_syncupSource);
177     ha.close();
178 
179     ha = new HBaseAdmin(conf2);
180     ha.createTable(t1_syncupTarget);
181     ha.createTable(t2_syncupTarget);
182     ha.close();
183 
184     // Get HTable from Master
185     ht1Source = new HTable(conf1, t1_su);
186     ht1Source.setWriteBufferSize(1024);
187     ht2Source = new HTable(conf1, t2_su);
188     ht1Source.setWriteBufferSize(1024);
189 
190     // Get HTable from Peer1
191     ht1TargetAtPeer1 = new HTable(conf2, t1_su);
192     ht1TargetAtPeer1.setWriteBufferSize(1024);
193     ht2TargetAtPeer1 = new HTable(conf2, t2_su);
194     ht2TargetAtPeer1.setWriteBufferSize(1024);
195 
196     /**
197      * set M-S : Master: utility1 Slave1: utility2
198      */
199     admin1.addPeer("1", utility2.getClusterKey());
200 
201     admin1.close();
202     admin2.close();
203   }
204 
205   private void putAndReplicateRows() throws Exception {
206     // add rows to Master cluster,
207     Put p;
208 
209     // 100 + 1 row to t1_syncup
210     for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
211       p = new Put(Bytes.toBytes("row" + i));
212       p.add(famName, qualName, Bytes.toBytes("val" + i));
213       ht1Source.put(p);
214     }
215     p = new Put(Bytes.toBytes("row" + 9999));
216     p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9999));
217     ht1Source.put(p);
218 
219     // 200 + 1 row to t2_syncup
220     for (int i = 0; i < NB_ROWS_IN_BATCH * 2; i++) {
221       p = new Put(Bytes.toBytes("row" + i));
222       p.add(famName, qualName, Bytes.toBytes("val" + i));
223       ht2Source.put(p);
224     }
225     p = new Put(Bytes.toBytes("row" + 9999));
226     p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9999));
227     ht2Source.put(p);
228 
229     // ensure replication completed
230     Thread.sleep(SLEEP_TIME);
231     int rowCount_ht1Source = utility1.countRows(ht1Source);
232     for (int i = 0; i < NB_RETRIES; i++) {
233       int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
234       if (i==NB_RETRIES-1) {
235         assertEquals("t1_syncup has 101 rows on source, and 100 on slave1", rowCount_ht1Source - 1,
236             rowCount_ht1TargetAtPeer1);
237       }
238       if (rowCount_ht1Source - 1 == rowCount_ht1TargetAtPeer1) {
239         break;
240       }
241       Thread.sleep(SLEEP_TIME);
242     }
243 
244     int rowCount_ht2Source = utility1.countRows(ht2Source);
245     for (int i = 0; i < NB_RETRIES; i++) {
246       int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
247       if (i==NB_RETRIES-1) {
248         assertEquals("t2_syncup has 201 rows on source, and 200 on slave1", rowCount_ht2Source - 1,
249             rowCount_ht2TargetAtPeer1);
250       }
251       if (rowCount_ht2Source - 1 == rowCount_ht2TargetAtPeer1) {
252         break;
253       }
254       Thread.sleep(SLEEP_TIME);
255     }
256   }
257 
258   private void mimicSyncUpAfterDelete() throws Exception {
259     utility2.shutdownMiniHBaseCluster();
260 
261     List<Delete> list = new ArrayList<Delete>();
262     // delete half of the rows
263     for (int i = 0; i < NB_ROWS_IN_BATCH / 2; i++) {
264       String rowKey = "row" + i;
265       Delete del = new Delete(rowKey.getBytes());
266       list.add(del);
267     }
268     ht1Source.delete(list);
269 
270     for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
271       String rowKey = "row" + i;
272       Delete del = new Delete(rowKey.getBytes());
273       list.add(del);
274     }
275     ht2Source.delete(list);
276 
277     int rowCount_ht1Source = utility1.countRows(ht1Source);
278     assertEquals("t1_syncup has 51 rows on source, after remove 50 of the replicated colfam", 51,
279       rowCount_ht1Source);
280 
281     int rowCount_ht2Source = utility1.countRows(ht2Source);
282     assertEquals("t2_syncup has 101 rows on source, after remove 100 of the replicated colfam",
283       101, rowCount_ht2Source);
284 
285     utility1.shutdownMiniHBaseCluster();
286     utility2.restartHBaseCluster(1);
287 
288     Thread.sleep(SLEEP_TIME);
289 
290     // before sync up
291     int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
292     int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
293     assertEquals("@Peer1 t1_syncup should still have 100 rows", 100, rowCount_ht1TargetAtPeer1);
294     assertEquals("@Peer1 t2_syncup should still have 200 rows", 200, rowCount_ht2TargetAtPeer1);
295 
296     // After sync up
297     syncUp(utility1);
298     rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
299     rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
300     assertEquals("@Peer1 t1_syncup should be sync up and have 50 rows", 50,
301       rowCount_ht1TargetAtPeer1);
302     assertEquals("@Peer1 t2_syncup should be sync up and have 100 rows", 100,
303       rowCount_ht2TargetAtPeer1);
304 
305   }
306 
307   private void mimicSyncUpAfterPut() throws Exception {
308     utility1.restartHBaseCluster(1);
309     utility2.shutdownMiniHBaseCluster();
310 
311     Put p;
312     // another 100 + 1 row to t1_syncup
313     // we should see 100 + 2 rows now
314     for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
315       p = new Put(Bytes.toBytes("row" + i));
316       p.add(famName, qualName, Bytes.toBytes("val" + i));
317       ht1Source.put(p);
318     }
319     p = new Put(Bytes.toBytes("row" + 9998));
320     p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9998));
321     ht1Source.put(p);
322 
323     // another 200 + 1 row to t1_syncup
324     // we should see 200 + 2 rows now
325     for (int i = 0; i < NB_ROWS_IN_BATCH * 2; i++) {
326       p = new Put(Bytes.toBytes("row" + i));
327       p.add(famName, qualName, Bytes.toBytes("val" + i));
328       ht2Source.put(p);
329     }
330     p = new Put(Bytes.toBytes("row" + 9998));
331     p.add(noRepfamName, qualName, Bytes.toBytes("val" + 9998));
332     ht2Source.put(p);
333 
334     int rowCount_ht1Source = utility1.countRows(ht1Source);
335     assertEquals("t1_syncup has 102 rows on source", 102, rowCount_ht1Source);
336     int rowCount_ht2Source = utility1.countRows(ht2Source);
337     assertEquals("t2_syncup has 202 rows on source", 202, rowCount_ht2Source);
338 
339     utility1.shutdownMiniHBaseCluster();
340     utility2.restartHBaseCluster(1);
341 
342     Thread.sleep(SLEEP_TIME);
343 
344     // before sync up
345     int rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
346     int rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
347     assertEquals("@Peer1 t1_syncup should be NOT sync up and have 50 rows", 50,
348       rowCount_ht1TargetAtPeer1);
349     assertEquals("@Peer1 t2_syncup should be NOT sync up and have 100 rows", 100,
350       rowCount_ht2TargetAtPeer1);
351 
352     // after syun up
353     syncUp(utility1);
354     rowCount_ht1TargetAtPeer1 = utility2.countRows(ht1TargetAtPeer1);
355     rowCount_ht2TargetAtPeer1 = utility2.countRows(ht2TargetAtPeer1);
356     assertEquals("@Peer1 t1_syncup should be sync up and have 100 rows", 100,
357       rowCount_ht1TargetAtPeer1);
358     assertEquals("@Peer1 t2_syncup should be sync up and have 200 rows", 200,
359       rowCount_ht2TargetAtPeer1);
360 
361   }
362 
363   private void syncUp(HBaseTestingUtility ut) throws Exception {
364     ReplicationSyncUp.setConfigure(ut.getConfiguration());
365     String[] arguments = new String[] { null };
366     new ReplicationSyncUp().run(arguments);
367   }
368 
369 }