1   /*
2    * Copyright 2011 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 java.io.Closeable;
27  import java.io.IOException;
28  import java.util.List;
29  import java.util.Random;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.hbase.*;
35  import org.apache.hadoop.hbase.client.Delete;
36  import org.apache.hadoop.hbase.client.Get;
37  import org.apache.hadoop.hbase.client.HBaseAdmin;
38  import org.apache.hadoop.hbase.client.HTable;
39  import org.apache.hadoop.hbase.client.Put;
40  import org.apache.hadoop.hbase.client.Result;
41  import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
42  import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
43  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
44  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
45  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
46  import org.apache.hadoop.hbase.util.Bytes;
47  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
48  import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
49  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
50  import org.junit.After;
51  import org.junit.Before;
52  import org.junit.Test;
53  import org.junit.experimental.categories.Category;
54  
55  @Category(LargeTests.class)
56  public class TestMasterReplication {
57  
58    private static final Log LOG = LogFactory.getLog(TestReplicationBase.class);
59  
60    private Configuration baseConfiguration;
61  
62    private HBaseTestingUtility[] utilities;
63    private Configuration[] configurations;
64    private MiniZooKeeperCluster miniZK;
65  
66    private static final long SLEEP_TIME = 500;
67    private static final int NB_RETRIES = 10;
68  
69    private static final byte[] tableName = Bytes.toBytes("test");
70    private static final byte[] famName = Bytes.toBytes("f");
71    private static final byte[] row = Bytes.toBytes("row");
72    private static final byte[] row1 = Bytes.toBytes("row1");
73    private static final byte[] row2 = Bytes.toBytes("row2");
74    private static final byte[] row3 = Bytes.toBytes("row3");
75    private static final byte[] row4 = Bytes.toBytes("row4");
76    private static final byte[] noRepfamName = Bytes.toBytes("norep");
77  
78    private static final byte[] count = Bytes.toBytes("count");
79    private static final byte[] put = Bytes.toBytes("put");
80    private static final byte[] delete = Bytes.toBytes("delete");
81  
82    private HTableDescriptor table;
83  
84    @Before
85    public void setUp() throws Exception {
86      baseConfiguration = HBaseConfiguration.create();
87      // smaller block size and capacity to trigger more operations
88      // and test them
89      baseConfiguration.setInt("hbase.regionserver.hlog.blocksize", 1024 * 20);
90      baseConfiguration.setInt("replication.source.size.capacity", 1024);
91      baseConfiguration.setLong("replication.source.sleepforretries", 100);
92      baseConfiguration.setInt("hbase.regionserver.maxlogs", 10);
93      baseConfiguration.setLong("hbase.master.logcleaner.ttl", 10);
94      baseConfiguration.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true);
95      baseConfiguration.setBoolean("dfs.support.append", true);
96      baseConfiguration.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100);
97      baseConfiguration.setStrings(
98          CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY,
99          CoprocessorCounter.class.getName());
100 
101     table = new HTableDescriptor(tableName);
102     HColumnDescriptor fam = new HColumnDescriptor(famName);
103     fam.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
104     table.addFamily(fam);
105     fam = new HColumnDescriptor(noRepfamName);
106     table.addFamily(fam);
107   }
108 
109   /**
110    * It tests the replication scenario involving 0 -> 1 -> 0. It does it by
111    * adding and deleting a row to a table in each cluster, checking if it's
112    * replicated. It also tests that the puts and deletes are not replicated back
113    * to the originating cluster.
114    */
115   @Test(timeout = 300000)
116   public void testCyclicReplication1() throws Exception {
117     LOG.info("testSimplePutDelete");
118     int numClusters = 2;
119     HTable[] htables = null;
120     try {
121       startMiniClusters(numClusters);
122       createTableOnClusters(table);
123 
124       htables = getHTablesOnClusters(tableName);
125 
126       // Test the replication scenarios of 0 -> 1 -> 0
127       addPeer("1", 0, 1);
128       addPeer("1", 1, 0);
129 
130       int[] expectedCounts = new int[] { 2, 2 };
131 
132       // add rows to both clusters,
133       // make sure they are both replication
134       putAndWait(row, famName, htables[0], htables[1]);
135       putAndWait(row1, famName, htables[1], htables[0]);
136       validateCounts(htables, put, expectedCounts);
137 
138       deleteAndWait(row, htables[0], htables[1]);
139       deleteAndWait(row1, htables[1], htables[0]);
140       validateCounts(htables, delete, expectedCounts);
141     } finally {
142       close(htables);
143       shutDownMiniClusters();
144     }
145   }
146 
147   /**
148    * Tests the cyclic replication scenario of 0 -> 1 -> 2 -> 0 by adding and
149    * deleting rows to a table in each clusters and ensuring that the each of
150    * these clusters get the appropriate mutations. It also tests the grouping
151    * scenario where a cluster needs to replicate the edits originating from
152    * itself and also the edits that it received using replication from a
153    * different cluster. The scenario is explained in HBASE-9158
154    */
155   @Test(timeout = 300000)
156   public void testCyclicReplication2() throws Exception {
157     LOG.info("testCyclicReplication1");
158     int numClusters = 3;
159     HTable[] htables = null;
160     try {
161       startMiniClusters(numClusters);
162       createTableOnClusters(table);
163 
164       // Test the replication scenario of 0 -> 1 -> 2 -> 0
165       addPeer("1", 0, 1);
166       addPeer("1", 1, 2);
167       addPeer("1", 2, 0);
168 
169       htables = getHTablesOnClusters(tableName);
170 
171       // put "row" and wait 'til it got around
172       putAndWait(row, famName, htables[0], htables[2]);
173       putAndWait(row1, famName, htables[1], htables[0]);
174       putAndWait(row2, famName, htables[2], htables[1]);
175 
176       deleteAndWait(row, htables[0], htables[2]);
177       deleteAndWait(row1, htables[1], htables[0]);
178       deleteAndWait(row2, htables[2], htables[1]);
179 
180       int[] expectedCounts = new int[] { 3, 3, 3 };
181       validateCounts(htables, put, expectedCounts);
182       validateCounts(htables, delete, expectedCounts);
183 
184       // Test HBASE-9158
185       disablePeer("1", 2);
186       // we now have an edit that was replicated into cluster originating from
187       // cluster 0
188       putAndWait(row3, famName, htables[0], htables[1]);
189       // now add a local edit to cluster 1
190       htables[1].put(new Put(row4).add(famName, row4, row4));
191       // re-enable replication from cluster 2 to cluster 0
192       enablePeer("1", 2);
193       // without HBASE-9158 the edit for row4 would have been marked with
194       // cluster 0's id
195       // and hence not replicated to cluster 0
196       wait(row4, htables[0], true);
197     } finally {
198       close(htables);
199       shutDownMiniClusters();
200     }
201   }
202 
203   /**
204    * Tests cyclic replication scenario of 0 -> 1 -> 2 -> 1.
205    */
206   @Test(timeout = 300000)
207   public void testCyclicReplication3() throws Exception {
208     LOG.info("testCyclicReplication2");
209     int numClusters = 3;
210     HTable[] htables = null;
211     try {
212       startMiniClusters(numClusters);
213       createTableOnClusters(table);
214 
215       // Test the replication scenario of 0 -> 1 -> 2 -> 1
216       addPeer("1", 0, 1);
217       addPeer("1", 1, 2);
218       addPeer("1", 2, 1);
219 
220       htables = getHTablesOnClusters(tableName);
221 
222       // put "row" and wait 'til it got around
223       putAndWait(row, famName, htables[0], htables[2]);
224       putAndWait(row1, famName, htables[1], htables[2]);
225       putAndWait(row2, famName, htables[2], htables[1]);
226 
227       deleteAndWait(row, htables[0], htables[2]);
228       deleteAndWait(row1, htables[1], htables[2]);
229       deleteAndWait(row2, htables[2], htables[1]);
230 
231       int[] expectedCounts = new int[] { 1, 3, 3 };
232       validateCounts(htables, put, expectedCounts);
233       validateCounts(htables, delete, expectedCounts);
234     } finally {
235       close(htables);
236       shutDownMiniClusters();
237     }
238   }
239 
240   @After
241   public void tearDown() throws IOException {
242     configurations = null;
243     utilities = null;
244   }
245 
246   @SuppressWarnings("resource")
247   private void startMiniClusters(int numClusters) throws Exception {
248     Random random = new Random();
249     utilities = new HBaseTestingUtility[numClusters];
250     configurations = new Configuration[numClusters];
251     for (int i = 0; i < numClusters; i++) {
252       Configuration conf = new Configuration(baseConfiguration);
253       conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/" + i + random.nextInt());
254       HBaseTestingUtility utility = new HBaseTestingUtility(conf);
255       if (i == 0) {
256         utility.startMiniZKCluster();
257         miniZK = utility.getZkCluster();
258       } else {
259         utility.setZkCluster(miniZK);
260       }
261       utility.startMiniCluster();
262       utilities[i] = utility;
263       configurations[i] = conf;
264       new ZooKeeperWatcher(conf, "cluster" + i, null, true);
265     }
266   }
267 
268   private void shutDownMiniClusters() throws Exception {
269     int numClusters = utilities.length;
270     for (int i = numClusters - 1; i >= 0; i--) {
271       if (utilities[i] != null) {
272         utilities[i].shutdownMiniCluster();
273       }
274     }
275     miniZK.shutdown();
276   }
277 
278   private void createTableOnClusters(HTableDescriptor table) throws Exception {
279     int numClusters = configurations.length;
280     for (int i = 0; i < numClusters; i++) {
281       HBaseAdmin hbaseAdmin = null;
282       try {
283         hbaseAdmin = new HBaseAdmin(configurations[i]);
284         hbaseAdmin.createTable(table);
285       } finally {
286         close(hbaseAdmin);
287       }
288     }
289   }
290 
291   private void addPeer(String id, int masterClusterNumber,
292       int slaveClusterNumber) throws Exception {
293     ReplicationAdmin replicationAdmin = null;
294     try {
295       replicationAdmin = new ReplicationAdmin(
296           configurations[masterClusterNumber]);
297       replicationAdmin.addPeer(id,
298           utilities[slaveClusterNumber].getClusterKey());
299     } finally {
300       close(replicationAdmin);
301     }
302   }
303 
304   private void disablePeer(String id, int masterClusterNumber) throws Exception {
305     ReplicationAdmin replicationAdmin = null;
306     try {
307       replicationAdmin = new ReplicationAdmin(
308           configurations[masterClusterNumber]);
309       replicationAdmin.disablePeer(id);
310     } finally {
311       close(replicationAdmin);
312     }
313   }
314 
315   private void enablePeer(String id, int masterClusterNumber) throws Exception {
316     ReplicationAdmin replicationAdmin = null;
317     try {
318       replicationAdmin = new ReplicationAdmin(
319           configurations[masterClusterNumber]);
320       replicationAdmin.enablePeer(id);
321     } finally {
322       close(replicationAdmin);
323     }
324   }
325 
326   private void close(Closeable... closeables) {
327     try {
328       if (closeables != null) {
329         for (Closeable closeable : closeables) {
330           closeable.close();
331         }
332       }
333     } catch (Exception e) {
334       LOG.warn("Exception occured while closing the object:", e);
335     }
336   }
337 
338   @SuppressWarnings("resource")
339   private HTable[] getHTablesOnClusters(byte[] tableName) throws Exception {
340     int numClusters = utilities.length;
341     HTable[] htables = new HTable[numClusters];
342     for (int i = 0; i < numClusters; i++) {
343       HTable htable = new HTable(configurations[i], tableName);
344       htable.setWriteBufferSize(1024);
345       htables[i] = htable;
346     }
347     return htables;
348   }
349 
350   private void validateCounts(HTable[] htables, byte[] type,
351       int[] expectedCounts) throws IOException {
352     for (int i = 0; i < htables.length; i++) {
353       assertEquals(Bytes.toString(type) + " were replicated back ",
354           expectedCounts[i], getCount(htables[i], type));
355     }
356   }
357 
358   private int getCount(HTable t, byte[] type) throws IOException {
359     Get test = new Get(row);
360     test.setAttribute("count", new byte[] {});
361     Result res = t.get(test);
362     return Bytes.toInt(res.getValue(count, type));
363   }
364 
365   private void deleteAndWait(byte[] row, HTable source, HTable target)
366       throws Exception {
367     Delete del = new Delete(row);
368     source.delete(del);
369     wait(row, target, true);
370   }
371 
372   private void putAndWait(byte[] row, byte[] fam, HTable source, HTable target)
373       throws Exception {
374     Put put = new Put(row);
375     put.add(fam, row, row);
376     source.put(put);
377     wait(row, target, false);
378   }
379 
380   private void wait(byte[] row, HTable target, boolean isDeleted)
381       throws Exception {
382     Get get = new Get(row);
383     for (int i = 0; i < NB_RETRIES; i++) {
384       if (i == NB_RETRIES - 1) {
385         fail("Waited too much time for replication. Row:" + Bytes.toString(row)
386             + ". IsDeleteReplication:" + isDeleted);
387       }
388       Result res = target.get(get);
389       boolean sleep = isDeleted ? res.size() > 0 : res.size() == 0;
390       if (sleep) {
391         LOG.info("Waiting for more time for replication. Row:"
392             + Bytes.toString(row) + ". IsDeleteReplication:" + isDeleted);
393         Thread.sleep(SLEEP_TIME);
394       } else {
395         if (!isDeleted) {
396           assertArrayEquals(res.value(), row);
397         }
398         LOG.info("Obtained row:"
399             + Bytes.toString(row) + ". IsDeleteReplication:" + isDeleted);
400         break;
401       }
402     }
403   }
404 
405   /**
406    * Use a coprocessor to count puts and deletes.
407    * as KVs would be replicated back with the same timestamp
408    * there is otherwise no way to count them.
409    */
410   public static class CoprocessorCounter extends BaseRegionObserver {
411     private int nCount = 0;
412     private int nDelete = 0;
413 
414     @Override
415     public void prePut(final ObserverContext<RegionCoprocessorEnvironment> e,
416         final Put put, final WALEdit edit, 
417         final boolean writeToWAL)
418         throws IOException {
419       nCount++;
420     }
421     @Override
422     public void postDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
423         final Delete delete, final WALEdit edit, 
424         final boolean writeToWAL)
425         throws IOException {
426       nDelete++;
427     }
428     @Override
429     public void preGet(final ObserverContext<RegionCoprocessorEnvironment> c,
430         final Get get, final List<KeyValue> result) throws IOException {
431       if (get.getAttribute("count") != null) {
432         result.clear();
433         // order is important!
434         result.add(new KeyValue(count, count, delete, Bytes.toBytes(nDelete)));
435         result.add(new KeyValue(count, count, put, Bytes.toBytes(nCount)));
436         c.bypass();
437       }
438     }
439   }
440 
441   @org.junit.Rule
442   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
443     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
444 }
445