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