1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.replication;
21  
22  import static org.junit.Assert.assertArrayEquals;
23  import static org.junit.Assert.assertEquals;
24  import static org.junit.Assert.fail;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.hbase.HBaseConfiguration;
30  import org.apache.hadoop.hbase.HBaseTestingUtility;
31  import org.apache.hadoop.hbase.HColumnDescriptor;
32  import org.apache.hadoop.hbase.HConstants;
33  import org.apache.hadoop.hbase.HTableDescriptor;
34  import org.apache.hadoop.hbase.KeyValue;
35  import org.apache.hadoop.hbase.UnknownScannerException;
36  import org.apache.hadoop.hbase.client.Delete;
37  import org.apache.hadoop.hbase.client.Get;
38  import org.apache.hadoop.hbase.client.HBaseAdmin;
39  import org.apache.hadoop.hbase.client.HTable;
40  import org.apache.hadoop.hbase.client.Put;
41  import org.apache.hadoop.hbase.client.Result;
42  import org.apache.hadoop.hbase.client.ResultScanner;
43  import org.apache.hadoop.hbase.client.Scan;
44  import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
45  import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication;
46  import org.apache.hadoop.hbase.util.Bytes;
47  import org.apache.hadoop.hbase.util.JVMClusterUtil;
48  import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
49  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
50  import org.apache.hadoop.mapreduce.Job;
51  import org.junit.AfterClass;
52  import org.junit.Before;
53  import org.junit.BeforeClass;
54  import org.junit.Test;
55  
56  public class TestReplication {
57  
58    private static final Log LOG = LogFactory.getLog(TestReplication.class);
59  
60    private static Configuration conf1;
61    private static Configuration conf2;
62  
63    private static ZooKeeperWatcher zkw1;
64    private static ZooKeeperWatcher zkw2;
65  
66    private static ReplicationAdmin admin;
67    private static String slaveClusterKey;
68  
69    private static HTable htable1;
70    private static HTable htable2;
71  
72    private static HBaseTestingUtility utility1;
73    private static HBaseTestingUtility utility2;
74    private static final int NB_ROWS_IN_BATCH = 100;
75    private static final int NB_ROWS_IN_BIG_BATCH =
76        NB_ROWS_IN_BATCH * 10;
77    private static final long SLEEP_TIME = 500;
78    private static final int NB_RETRIES = 10;
79  
80    private static final byte[] tableName = Bytes.toBytes("test");
81    private static final byte[] famName = Bytes.toBytes("f");
82    private static final byte[] row = Bytes.toBytes("row");
83    private static final byte[] noRepfamName = Bytes.toBytes("norep");
84  
85    /**
86     * @throws java.lang.Exception
87     */
88    @BeforeClass
89    public static void setUpBeforeClass() throws Exception {
90      conf1 = HBaseConfiguration.create();
91      conf1.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
92      // smaller block size and capacity to trigger more operations
93      // and test them
94      conf1.setInt("hbase.regionserver.hlog.blocksize", 1024*20);
95      conf1.setInt("replication.source.size.capacity", 1024);
96      conf1.setLong("replication.source.sleepforretries", 100);
97      conf1.setInt("hbase.regionserver.maxlogs", 10);
98      conf1.setLong("hbase.master.logcleaner.ttl", 10);
99      conf1.setLong("hbase.client.retries.number", 5);
100     conf1.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true);
101     conf1.setBoolean("dfs.support.append", true);
102     conf1.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
103 
104     utility1 = new HBaseTestingUtility(conf1);
105     utility1.startMiniZKCluster();
106     MiniZooKeeperCluster miniZK = utility1.getZkCluster();
107     zkw1 = new ZooKeeperWatcher(conf1, "cluster1", null);
108     admin = new ReplicationAdmin(conf1);
109     LOG.info("Setup first Zk");
110 
111     conf2 = HBaseConfiguration.create();
112     conf2.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/2");
113     conf2.setInt("hbase.client.retries.number", 6);
114     conf2.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true);
115     conf2.setBoolean("dfs.support.append", true);
116 
117     utility2 = new HBaseTestingUtility(conf2);
118     utility2.setZkCluster(miniZK);
119     zkw2 = new ZooKeeperWatcher(conf2, "cluster2", null);
120 
121     slaveClusterKey = conf2.get(HConstants.ZOOKEEPER_QUORUM)+":" +
122             conf2.get("hbase.zookeeper.property.clientPort")+":/2";
123     admin.addPeer("2", slaveClusterKey);
124     setIsReplication(true);
125 
126     LOG.info("Setup second Zk");
127 
128     utility1.startMiniCluster(2);
129     utility2.startMiniCluster(2);
130 
131     HTableDescriptor table = new HTableDescriptor(tableName);
132     HColumnDescriptor fam = new HColumnDescriptor(famName);
133     fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
134     table.addFamily(fam);
135     fam = new HColumnDescriptor(noRepfamName);
136     table.addFamily(fam);
137     HBaseAdmin admin1 = new HBaseAdmin(conf1);
138     HBaseAdmin admin2 = new HBaseAdmin(conf2);
139     admin1.createTable(table);
140     admin2.createTable(table);
141 
142     htable1 = new HTable(conf1, tableName);
143     htable1.setWriteBufferSize(1024);
144     htable2 = new HTable(conf2, tableName);
145   }
146 
147   private static void setIsReplication(boolean rep) throws Exception {
148     LOG.info("Set rep " + rep);
149     admin.setReplicating(rep);
150     Thread.sleep(SLEEP_TIME);
151   }
152 
153   /**
154    * @throws java.lang.Exception
155    */
156   @Before
157   public void setUp() throws Exception {
158 
159     // Starting and stopping replication can make us miss new logs,
160     // rolling like this makes sure the most recent one gets added to the queue
161     for ( JVMClusterUtil.RegionServerThread r :
162         utility1.getHBaseCluster().getRegionServerThreads()) {
163       r.getRegionServer().getWAL().rollWriter();
164     }
165     utility1.truncateTable(tableName);
166     // truncating the table will send one Delete per row to the slave cluster
167     // in an async fashion, which is why we cannot just call truncateTable on
168     // utility2 since late writes could make it to the slave in some way.
169     // Instead, we truncate the first table and wait for all the Deletes to
170     // make it to the slave.
171     Scan scan = new Scan();
172     int lastCount = 0;
173     for (int i = 0; i < NB_RETRIES; i++) {
174       if (i==NB_RETRIES-1) {
175         fail("Waited too much time for truncate");
176       }
177       ResultScanner scanner = htable2.getScanner(scan);
178       Result[] res = scanner.next(NB_ROWS_IN_BIG_BATCH);
179       scanner.close();
180       if (res.length != 0) {
181        if (res.length < lastCount) {
182           i--; // Don't increment timeout if we make progress
183         }
184         lastCount = res.length;
185         LOG.info("Still got " + res.length + " rows");
186         Thread.sleep(SLEEP_TIME);
187       } else {
188         break;
189       }
190     }
191   }
192 
193   /**
194    * @throws java.lang.Exception
195    */
196   @AfterClass
197   public static void tearDownAfterClass() throws Exception {
198     utility2.shutdownMiniCluster();
199     utility1.shutdownMiniCluster();
200   }
201 
202   /**
203    * Add a row, check it's replicated, delete it, check's gone
204    * @throws Exception
205    */
206   @Test
207   public void testSimplePutDelete() throws Exception {
208     LOG.info("testSimplePutDelete");
209     Put put = new Put(row);
210     put.add(famName, row, row);
211 
212     htable1 = new HTable(conf1, tableName);
213     htable1.put(put);
214 
215     Get get = new Get(row);
216     for (int i = 0; i < NB_RETRIES; i++) {
217       if (i==NB_RETRIES-1) {
218         fail("Waited too much time for put replication");
219       }
220       Result res = htable2.get(get);
221       if (res.size() == 0) {
222         LOG.info("Row not available");
223         Thread.sleep(SLEEP_TIME);
224       } else {
225         assertArrayEquals(res.value(), row);
226         break;
227       }
228     }
229 
230     Delete del = new Delete(row);
231     htable1.delete(del);
232 
233     get = new Get(row);
234     for (int i = 0; i < NB_RETRIES; i++) {
235       if (i==NB_RETRIES-1) {
236         fail("Waited too much time for del replication");
237       }
238       Result res = htable2.get(get);
239       if (res.size() >= 1) {
240         LOG.info("Row not deleted");
241         Thread.sleep(SLEEP_TIME);
242       } else {
243         break;
244       }
245     }
246   }
247 
248   /**
249    * Try a small batch upload using the write buffer, check it's replicated
250    * @throws Exception
251    */
252   @Test
253   public void testSmallBatch() throws Exception {
254     LOG.info("testSmallBatch");
255     Put put;
256     // normal Batch tests
257     htable1.setAutoFlush(false);
258     for (int i = 0; i < NB_ROWS_IN_BATCH; i++) {
259       put = new Put(Bytes.toBytes(i));
260       put.add(famName, row, row);
261       htable1.put(put);
262     }
263     htable1.flushCommits();
264 
265     Scan scan = new Scan();
266 
267     ResultScanner scanner1 = htable1.getScanner(scan);
268     Result[] res1 = scanner1.next(NB_ROWS_IN_BATCH);
269     scanner1.close();
270     assertEquals(NB_ROWS_IN_BATCH, res1.length);
271 
272     for (int i = 0; i < NB_RETRIES; i++) {
273       if (i==NB_RETRIES-1) {
274         fail("Waited too much time for normal batch replication");
275       }
276       ResultScanner scanner = htable2.getScanner(scan);
277       Result[] res = scanner.next(NB_ROWS_IN_BATCH);
278       scanner.close();
279       if (res.length != NB_ROWS_IN_BATCH) {
280         LOG.info("Only got " + res.length + " rows");
281         Thread.sleep(SLEEP_TIME);
282       } else {
283         break;
284       }
285     }
286 
287     htable1.setAutoFlush(true);
288 
289   }
290 
291   /**
292    * Test stopping replication, trying to insert, make sure nothing's
293    * replicated, enable it, try replicating and it should work
294    * @throws Exception
295    */
296   @Test
297   public void testStartStop() throws Exception {
298 
299     // Test stopping replication
300     setIsReplication(false);
301 
302     Put put = new Put(Bytes.toBytes("stop start"));
303     put.add(famName, row, row);
304     htable1.put(put);
305 
306     Get get = new Get(Bytes.toBytes("stop start"));
307     for (int i = 0; i < NB_RETRIES; i++) {
308       if (i==NB_RETRIES-1) {
309         break;
310       }
311       Result res = htable2.get(get);
312       if(res.size() >= 1) {
313         fail("Replication wasn't stopped");
314 
315       } else {
316         LOG.info("Row not replicated, let's wait a bit more...");
317         Thread.sleep(SLEEP_TIME);
318       }
319     }
320 
321     // Test restart replication
322     setIsReplication(true);
323 
324     htable1.put(put);
325 
326     for (int i = 0; i < NB_RETRIES; i++) {
327       if (i==NB_RETRIES-1) {
328         fail("Waited too much time for put replication");
329       }
330       Result res = htable2.get(get);
331       if(res.size() == 0) {
332         LOG.info("Row not available");
333         Thread.sleep(SLEEP_TIME);
334       } else {
335         assertArrayEquals(res.value(), row);
336         break;
337       }
338     }
339 
340     put = new Put(Bytes.toBytes("do not rep"));
341     put.add(noRepfamName, row, row);
342     htable1.put(put);
343 
344     get = new Get(Bytes.toBytes("do not rep"));
345     for (int i = 0; i < NB_RETRIES; i++) {
346       if (i == NB_RETRIES-1) {
347         break;
348       }
349       Result res = htable2.get(get);
350       if (res.size() >= 1) {
351         fail("Not supposed to be replicated");
352       } else {
353         LOG.info("Row not replicated, let's wait a bit more...");
354         Thread.sleep(SLEEP_TIME);
355       }
356     }
357 
358   }
359 
360   /**
361    * Integration test for TestReplicationAdmin, removes and re-add a peer
362    * cluster
363    * @throws Exception
364    */
365   @Test
366   public void testAddAndRemoveClusters() throws Exception {
367     LOG.info("testAddAndRemoveClusters");
368     admin.removePeer("2");
369     Thread.sleep(SLEEP_TIME);
370     byte[] rowKey = Bytes.toBytes("Won't be replicated");
371     Put put = new Put(rowKey);
372     put.add(famName, row, row);
373     htable1.put(put);
374 
375     Get get = new Get(rowKey);
376     for (int i = 0; i < NB_RETRIES; i++) {
377       if (i == NB_RETRIES-1) {
378         break;
379       }
380       Result res = htable2.get(get);
381       if (res.size() >= 1) {
382         fail("Not supposed to be replicated");
383       } else {
384         LOG.info("Row not replicated, let's wait a bit more...");
385         Thread.sleep(SLEEP_TIME);
386       }
387     }
388 
389     admin.addPeer("2", slaveClusterKey);
390     Thread.sleep(SLEEP_TIME);
391     rowKey = Bytes.toBytes("do rep");
392     put = new Put(rowKey);
393     put.add(famName, row, row);
394     LOG.info("Adding new row");
395     htable1.put(put);
396 
397     get = new Get(rowKey);
398     for (int i = 0; i < NB_RETRIES; i++) {
399       if (i==NB_RETRIES-1) {
400         fail("Waited too much time for put replication");
401       }
402       Result res = htable2.get(get);
403       if (res.size() == 0) {
404         LOG.info("Row not available");
405         Thread.sleep(SLEEP_TIME*i);
406       } else {
407         assertArrayEquals(res.value(), row);
408         break;
409       }
410     }
411   }
412 
413   /**
414    * Do a more intense version testSmallBatch, one  that will trigger
415    * hlog rolling and other non-trivial code paths
416    * @throws Exception
417    */
418   @Test
419   public void loadTesting() throws Exception {
420     htable1.setWriteBufferSize(1024);
421     htable1.setAutoFlush(false);
422     for (int i = 0; i < NB_ROWS_IN_BIG_BATCH; i++) {
423       Put put = new Put(Bytes.toBytes(i));
424       put.add(famName, row, row);
425       htable1.put(put);
426     }
427     htable1.flushCommits();
428 
429     Scan scan = new Scan();
430 
431     ResultScanner scanner = htable1.getScanner(scan);
432     Result[] res = scanner.next(NB_ROWS_IN_BIG_BATCH);
433     scanner.close();
434 
435     assertEquals(NB_ROWS_IN_BATCH *10, res.length);
436 
437     scan = new Scan();
438 
439     for (int i = 0; i < NB_RETRIES; i++) {
440 
441       scanner = htable2.getScanner(scan);
442       res = scanner.next(NB_ROWS_IN_BIG_BATCH);
443       scanner.close();
444       if (res.length != NB_ROWS_IN_BIG_BATCH) {
445         if (i == NB_RETRIES-1) {
446           int lastRow = -1;
447           for (Result result : res) {
448             int currentRow = Bytes.toInt(result.getRow());
449             for (int row = lastRow+1; row < currentRow; row++) {
450               LOG.error("Row missing: " + row);
451             }
452             lastRow = currentRow;
453           }
454           LOG.error("Last row: " + lastRow);
455           fail("Waited too much time for normal batch replication, "
456               + res.length + " instead of " + NB_ROWS_IN_BIG_BATCH);
457         } else {
458           LOG.info("Only got " + res.length + " rows");
459           Thread.sleep(SLEEP_TIME);
460         }
461       } else {
462         break;
463       }
464     }
465   }
466 
467   /**
468    * Do a small loading into a table, make sure the data is really the same,
469    * then run the VerifyReplication job to check the results. Do a second
470    * comparison where all the cells are different.
471    * @throws Exception
472    */
473   @Test
474   public void testVerifyRepJob() throws Exception {
475     // Populate the tables, at the same time it guarantees that the tables are
476     // identical since it does the check
477     testSmallBatch();
478 
479     String[] args = new String[] {"2", Bytes.toString(tableName)};
480     Job job = VerifyReplication.createSubmittableJob(conf1, args);
481     if (job == null) {
482       fail("Job wasn't created, see the log");
483     }
484     if (!job.waitForCompletion(true)) {
485       fail("Job failed, see the log");
486     }
487     assertEquals(NB_ROWS_IN_BATCH, job.getCounters().
488         findCounter(VerifyReplication.Verifier.Counters.GOODROWS).getValue());
489     assertEquals(0, job.getCounters().
490         findCounter(VerifyReplication.Verifier.Counters.BADROWS).getValue());
491 
492     Scan scan = new Scan();
493     ResultScanner rs = htable2.getScanner(scan);
494     Put put = null;
495     for (Result result : rs) {
496       put = new Put(result.getRow());
497       KeyValue firstVal = result.raw()[0];
498       put.add(firstVal.getFamily(),
499           firstVal.getQualifier(), Bytes.toBytes("diff data"));
500       htable2.put(put);
501     }
502     Delete delete = new Delete(put.getRow());
503     htable2.delete(delete);
504     job = VerifyReplication.createSubmittableJob(conf1, args);
505     if (job == null) {
506       fail("Job wasn't created, see the log");
507     }
508     if (!job.waitForCompletion(true)) {
509       fail("Job failed, see the log");
510     }
511     assertEquals(0, job.getCounters().
512             findCounter(VerifyReplication.Verifier.Counters.GOODROWS).getValue());
513         assertEquals(NB_ROWS_IN_BATCH, job.getCounters().
514             findCounter(VerifyReplication.Verifier.Counters.BADROWS).getValue());
515   }
516 
517   /**
518    * Load up multiple tables over 2 region servers and kill a source during
519    * the upload. The failover happens internally.
520    *
521    * WARNING this test sometimes fails because of HBASE-3515
522    *
523    * @throws Exception
524    */
525   @Test
526   public void queueFailover() throws Exception {
527     utility1.createMultiRegions(htable1, famName);
528 
529     // killing the RS with .META. can result into failed puts until we solve
530     // IO fencing
531     int rsToKill1 =
532         utility1.getHBaseCluster().getServerWithMeta() == 0 ? 1 : 0;
533     int rsToKill2 =
534         utility2.getHBaseCluster().getServerWithMeta() == 0 ? 1 : 0;
535 
536     // Takes about 20 secs to run the full loading, kill around the middle
537     Thread killer1 = killARegionServer(utility1, 7500, rsToKill1);
538     Thread killer2 = killARegionServer(utility2, 10000, rsToKill2);
539 
540     LOG.info("Start loading table");
541     int initialCount = utility1.loadTable(htable1, famName);
542     LOG.info("Done loading table");
543     killer1.join(5000);
544     killer2.join(5000);
545     LOG.info("Done waiting for threads");
546 
547     Result[] res;
548     while (true) {
549       try {
550         Scan scan = new Scan();
551         ResultScanner scanner = htable1.getScanner(scan);
552         res = scanner.next(initialCount);
553         scanner.close();
554         break;
555       } catch (UnknownScannerException ex) {
556         LOG.info("Cluster wasn't ready yet, restarting scanner");
557       }
558     }
559     // Test we actually have all the rows, we may miss some because we
560     // don't have IO fencing.
561     if (res.length != initialCount) {
562       LOG.warn("We lost some rows on the master cluster!");
563       // We don't really expect the other cluster to have more rows
564       initialCount = res.length;
565     }
566 
567     Scan scan2 = new Scan();
568 
569     int lastCount = 0;
570 
571     for (int i = 0; i < NB_RETRIES; i++) {
572       if (i==NB_RETRIES-1) {
573         fail("Waited too much time for queueFailover replication");
574       }
575       ResultScanner scanner2 = htable2.getScanner(scan2);
576       Result[] res2 = scanner2.next(initialCount * 2);
577       scanner2.close();
578       if (res2.length < initialCount) {
579         if (lastCount < res2.length) {
580           i--; // Don't increment timeout if we make progress
581         }
582         lastCount = res2.length;
583         LOG.info("Only got " + lastCount + " rows instead of " +
584             initialCount + " current i=" + i);
585         Thread.sleep(SLEEP_TIME*2);
586       } else {
587         break;
588       }
589     }
590   }
591 
592   private static Thread killARegionServer(final HBaseTestingUtility utility,
593                                    final long timeout, final int rs) {
594     Thread killer = new Thread() {
595       public void run() {
596         try {
597           Thread.sleep(timeout);
598           utility.expireRegionServerSession(rs);
599         } catch (Exception e) {
600           LOG.error(e);
601         }
602       }
603     };
604     killer.start();
605     return killer;
606   }
607 }