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