View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.replication.regionserver;
20  
21  import static org.junit.Assert.assertEquals;
22  import static org.junit.Assert.assertFalse;
23  import static org.junit.Assert.assertNull;
24  import static org.junit.Assert.assertTrue;
25  
26  import java.io.IOException;
27  import java.net.URLEncoder;
28  import java.util.ArrayList;
29  import java.util.Collection;
30  import java.util.HashMap;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.NavigableMap;
34  import java.util.SortedMap;
35  import java.util.SortedSet;
36  import java.util.TreeSet;
37  import java.util.UUID;
38  import java.util.concurrent.CountDownLatch;
39  import java.util.concurrent.atomic.AtomicLong;
40  
41  import org.apache.commons.logging.Log;
42  import org.apache.commons.logging.LogFactory;
43  import org.apache.hadoop.conf.Configuration;
44  import org.apache.hadoop.fs.FileSystem;
45  import org.apache.hadoop.fs.Path;
46  import org.apache.hadoop.hbase.ChoreService;
47  import org.apache.hadoop.hbase.ClusterId;
48  import org.apache.hadoop.hbase.CoordinatedStateManager;
49  import org.apache.hadoop.hbase.HBaseConfiguration;
50  import org.apache.hadoop.hbase.HBaseTestingUtility;
51  import org.apache.hadoop.hbase.HColumnDescriptor;
52  import org.apache.hadoop.hbase.HConstants;
53  import org.apache.hadoop.hbase.HRegionInfo;
54  import org.apache.hadoop.hbase.HTableDescriptor;
55  import org.apache.hadoop.hbase.KeyValue;
56  import org.apache.hadoop.hbase.Server;
57  import org.apache.hadoop.hbase.ServerName;
58  import org.apache.hadoop.hbase.TableName;
59  import org.apache.hadoop.hbase.client.ClusterConnection;
60  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
61  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
62  import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
63  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
64  import org.apache.hadoop.hbase.replication.ReplicationFactory;
65  import org.apache.hadoop.hbase.replication.ReplicationPeers;
66  import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
67  import org.apache.hadoop.hbase.replication.ReplicationQueues;
68  import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
69  import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
70  import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
71  import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
72  import org.apache.hadoop.hbase.testclassification.MediumTests;
73  import org.apache.hadoop.hbase.util.ByteStringer;
74  import org.apache.hadoop.hbase.util.Bytes;
75  import org.apache.hadoop.hbase.util.FSUtils;
76  import org.apache.hadoop.hbase.wal.WAL;
77  import org.apache.hadoop.hbase.wal.WALFactory;
78  import org.apache.hadoop.hbase.wal.WALKey;
79  import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
80  import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
81  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
82  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
83  import org.junit.After;
84  import org.junit.AfterClass;
85  import org.junit.Before;
86  import org.junit.BeforeClass;
87  import org.junit.Test;
88  import org.junit.experimental.categories.Category;
89  
90  import com.google.common.collect.Sets;
91  
92  @Category(MediumTests.class)
93  public class TestReplicationSourceManager {
94  
95    private static final Log LOG =
96        LogFactory.getLog(TestReplicationSourceManager.class);
97  
98    private static Configuration conf;
99  
100   private static HBaseTestingUtility utility;
101 
102   private static Replication replication;
103 
104   private static ReplicationSourceManager manager;
105 
106   private static ZooKeeperWatcher zkw;
107 
108   private static HTableDescriptor htd;
109 
110   private static HRegionInfo hri;
111 
112   private static final byte[] r1 = Bytes.toBytes("r1");
113 
114   private static final byte[] r2 = Bytes.toBytes("r2");
115 
116   private static final byte[] f1 = Bytes.toBytes("f1");
117 
118   private static final byte[] f2 = Bytes.toBytes("f2");
119 
120   private static final TableName test =
121       TableName.valueOf("test");
122 
123   private static final String slaveId = "1";
124 
125   private static FileSystem fs;
126 
127   private static Path oldLogDir;
128 
129   private static Path logDir;
130 
131   private static CountDownLatch latch;
132 
133   private static List<String> files = new ArrayList<String>();
134 
135   @BeforeClass
136   public static void setUpBeforeClass() throws Exception {
137 
138     conf = HBaseConfiguration.create();
139     conf.set("replication.replicationsource.implementation",
140         ReplicationSourceDummy.class.getCanonicalName());
141     conf.setBoolean(HConstants.REPLICATION_ENABLE_KEY,
142         HConstants.REPLICATION_ENABLE_DEFAULT);
143     conf.setLong("replication.sleep.before.failover", 2000);
144     conf.setInt("replication.source.maxretriesmultiplier", 10);
145     utility = new HBaseTestingUtility(conf);
146     utility.startMiniZKCluster();
147 
148     zkw = new ZooKeeperWatcher(conf, "test", null);
149     ZKUtil.createWithParents(zkw, "/hbase/replication");
150     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1");
151     ZKUtil.setData(zkw, "/hbase/replication/peers/1",
152         Bytes.toBytes(conf.get(HConstants.ZOOKEEPER_QUORUM) + ":"
153             + conf.get(HConstants.ZOOKEEPER_CLIENT_PORT) + ":/1"));
154     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state");
155     ZKUtil.setData(zkw, "/hbase/replication/peers/1/peer-state",
156       ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
157     ZKUtil.createWithParents(zkw, "/hbase/replication/state");
158     ZKUtil.setData(zkw, "/hbase/replication/state", ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
159 
160     ZKClusterId.setClusterId(zkw, new ClusterId());
161     FSUtils.setRootDir(utility.getConfiguration(), utility.getDataTestDir());
162     fs = FileSystem.get(conf);
163     oldLogDir = new Path(utility.getDataTestDir(),
164         HConstants.HREGION_OLDLOGDIR_NAME);
165     logDir = new Path(utility.getDataTestDir(),
166         HConstants.HREGION_LOGDIR_NAME);
167     replication = new Replication(new DummyServer(), fs, logDir, oldLogDir);
168     manager = replication.getReplicationManager();
169 
170     manager.addSource(slaveId);
171 
172     htd = new HTableDescriptor(test);
173     HColumnDescriptor col = new HColumnDescriptor(f1);
174     col.setScope(HConstants.REPLICATION_SCOPE_GLOBAL);
175     htd.addFamily(col);
176     col = new HColumnDescriptor(f2);
177     col.setScope(HConstants.REPLICATION_SCOPE_LOCAL);
178     htd.addFamily(col);
179 
180     hri = new HRegionInfo(htd.getTableName(), r1, r2);
181   }
182 
183   @AfterClass
184   public static void tearDownAfterClass() throws Exception {
185     manager.join();
186     utility.shutdownMiniCluster();
187   }
188 
189   @Before
190   public void setUp() throws Exception {
191     fs.delete(logDir, true);
192     fs.delete(oldLogDir, true);
193   }
194 
195   @After
196   public void tearDown() throws Exception {
197     setUp();
198   }
199 
200   @Test
201   public void testLogRoll() throws Exception {
202     long seq = 0;
203     long baseline = 1000;
204     long time = baseline;
205     KeyValue kv = new KeyValue(r1, f1, r1);
206     WALEdit edit = new WALEdit();
207     edit.add(kv);
208 
209     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
210     listeners.add(replication);
211     final WALFactory wals = new WALFactory(utility.getConfiguration(), listeners,
212         URLEncoder.encode("regionserver:60020", "UTF8"));
213     final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes());
214     final AtomicLong sequenceId = new AtomicLong(1);
215     manager.init();
216     HTableDescriptor htd = new HTableDescriptor();
217     htd.addFamily(new HColumnDescriptor(f1));
218     // Testing normal log rolling every 20
219     for(long i = 1; i < 101; i++) {
220       if(i > 1 && i % 20 == 0) {
221         wal.rollWriter();
222       }
223       LOG.info(i);
224       final long txid = wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), test,
225           System.currentTimeMillis()), edit, sequenceId, true ,null);
226       wal.sync(txid);
227     }
228 
229     // Simulate a rapid insert that's followed
230     // by a report that's still not totally complete (missing last one)
231     LOG.info(baseline + " and " + time);
232     baseline += 101;
233     time = baseline;
234     LOG.info(baseline + " and " + time);
235 
236     for (int i = 0; i < 3; i++) {
237       wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), test,
238           System.currentTimeMillis()), edit, sequenceId, true, null);
239     }
240     wal.sync();
241 
242     assertEquals(6, manager.getWALs().get(slaveId).size());
243 
244     wal.rollWriter();
245 
246     manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(),
247         "1", 0, false, false);
248 
249     wal.append(htd, hri, new WALKey(hri.getEncodedNameAsBytes(), test,
250         System.currentTimeMillis()), edit, sequenceId, true, null);
251     wal.sync();
252 
253     assertEquals(1, manager.getWALs().size());
254 
255 
256     // TODO Need a case with only 2 WALs and we only want to delete the first one
257   }
258 
259   @Test
260   public void testClaimQueues() throws Exception {
261     LOG.debug("testNodeFailoverWorkerCopyQueuesFromRSUsingMulti");
262     conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
263     final Server server = new DummyServer("hostname0.example.org");
264     ReplicationQueues rq =
265         ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
266           server);
267     rq.init(server.getServerName().toString());
268     // populate some znodes in the peer znode
269     files.add("log1");
270     files.add("log2");
271     for (String file : files) {
272       rq.addLog("1", file);
273     }
274     // create 3 DummyServers
275     Server s1 = new DummyServer("dummyserver1.example.org");
276     Server s2 = new DummyServer("dummyserver2.example.org");
277     Server s3 = new DummyServer("dummyserver3.example.org");
278 
279     // create 3 DummyNodeFailoverWorkers
280     DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(
281         server.getServerName().getServerName(), s1);
282     DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(
283         server.getServerName().getServerName(), s2);
284     DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(
285         server.getServerName().getServerName(), s3);
286 
287     latch = new CountDownLatch(3);
288     // start the threads
289     w1.start();
290     w2.start();
291     w3.start();
292     // make sure only one is successful
293     int populatedMap = 0;
294     // wait for result now... till all the workers are done.
295     latch.await();
296     populatedMap += w1.isLogZnodesMapPopulated() + w2.isLogZnodesMapPopulated()
297         + w3.isLogZnodesMapPopulated();
298     assertEquals(1, populatedMap);
299     server.abort("", null);
300   }
301 
302   @Test
303   public void testCleanupFailoverQueues() throws Exception {
304     final Server server = new DummyServer("hostname1.example.org");
305     ReplicationQueues rq =
306         ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
307           server);
308     rq.init(server.getServerName().toString());
309     // populate some znodes in the peer znode
310     SortedSet<String> files = new TreeSet<String>();
311     files.add("log1");
312     files.add("log2");
313     for (String file : files) {
314       rq.addLog("1", file);
315     }
316     Server s1 = new DummyServer("dummyserver1.example.org");
317     ReplicationQueues rq1 =
318         ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
319     rq1.init(s1.getServerName().toString());
320     ReplicationPeers rp1 =
321         ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration(), s1);
322     rp1.init();
323     NodeFailoverWorker w1 =
324         manager.new NodeFailoverWorker(server.getServerName().getServerName(), rq1, rp1, new UUID(
325             new Long(1), new Long(2)));
326     w1.start();
327     w1.join(5000);
328     assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
329     String id = "1-" + server.getServerName().getServerName();
330     assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id));
331     manager.cleanOldLogs("log2", id, true);
332     // log1 should be deleted
333     assertEquals(Sets.newHashSet("log2"), manager.getWalsByIdRecoveredQueues().get(id));
334   }
335 
336   @Test
337   public void testNodeFailoverDeadServerParsing() throws Exception {
338     LOG.debug("testNodeFailoverDeadServerParsing");
339     conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
340     final Server server = new DummyServer("ec2-54-234-230-108.compute-1.amazonaws.com");
341     ReplicationQueues repQueues =
342         ReplicationFactory.getReplicationQueues(server.getZooKeeper(), conf, server);
343     repQueues.init(server.getServerName().toString());
344     // populate some znodes in the peer znode
345     files.add("log1");
346     files.add("log2");
347     for (String file : files) {
348       repQueues.addLog("1", file);
349     }
350 
351     // create 3 DummyServers
352     Server s1 = new DummyServer("ip-10-8-101-114.ec2.internal");
353     Server s2 = new DummyServer("ec2-107-20-52-47.compute-1.amazonaws.com");
354     Server s3 = new DummyServer("ec2-23-20-187-167.compute-1.amazonaws.com");
355 
356     // simulate three servers fail sequentially
357     ReplicationQueues rq1 =
358         ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
359     rq1.init(s1.getServerName().toString());
360     SortedMap<String, SortedSet<String>> testMap =
361         rq1.claimQueues(server.getServerName().getServerName());
362     ReplicationQueues rq2 =
363         ReplicationFactory.getReplicationQueues(s2.getZooKeeper(), s2.getConfiguration(), s2);
364     rq2.init(s2.getServerName().toString());
365     testMap = rq2.claimQueues(s1.getServerName().getServerName());
366     ReplicationQueues rq3 =
367         ReplicationFactory.getReplicationQueues(s3.getZooKeeper(), s3.getConfiguration(), s3);
368     rq3.init(s3.getServerName().toString());
369     testMap = rq3.claimQueues(s2.getServerName().getServerName());
370 
371     ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(testMap.firstKey());
372     List<String> result = replicationQueueInfo.getDeadRegionServers();
373 
374     // verify
375     assertTrue(result.contains(server.getServerName().getServerName()));
376     assertTrue(result.contains(s1.getServerName().getServerName()));
377     assertTrue(result.contains(s2.getServerName().getServerName()));
378 
379     server.abort("", null);
380   }
381 
382   @Test
383   public void testFailoverDeadServerCversionChange() throws Exception {
384     LOG.debug("testFailoverDeadServerCversionChange");
385 
386     conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
387     final Server s0 = new DummyServer("cversion-change0.example.org");
388     ReplicationQueues repQueues =
389         ReplicationFactory.getReplicationQueues(s0.getZooKeeper(), conf, s0);
390     repQueues.init(s0.getServerName().toString());
391     // populate some znodes in the peer znode
392     files.add("log1");
393     files.add("log2");
394     for (String file : files) {
395       repQueues.addLog("1", file);
396     }
397     // simulate queue transfer
398     Server s1 = new DummyServer("cversion-change1.example.org");
399     ReplicationQueues rq1 =
400         ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
401     rq1.init(s1.getServerName().toString());
402 
403     ReplicationQueuesClient client =
404         ReplicationFactory.getReplicationQueuesClient(s1.getZooKeeper(), s1.getConfiguration(), s1);
405 
406     int v0 = client.getQueuesZNodeCversion();
407     rq1.claimQueues(s0.getServerName().getServerName());
408     int v1 = client.getQueuesZNodeCversion();
409     // cversion should increased by 1 since a child node is deleted
410     assertEquals(v0 + 1, v1);
411 
412     s0.abort("", null);
413   }
414 
415   @Test
416   public void testBulkLoadWALEditsWithoutBulkLoadReplicationEnabled() throws Exception {
417     // 1. Create wal key
418     WALKey logKey = new WALKey();
419     // 2. Get the bulk load wal edit event
420     WALEdit logEdit = getBulkLoadWALEdit();
421 
422     // 3. Get the scopes for the key
423     Replication.scopeWALEdits(htd, logKey, logEdit, conf, manager);
424 
425     // 4. Assert that no bulk load entry scopes are added if bulk load hfile replication is disabled
426     assertNull("No bulk load entries scope should be added if bulk load replication is diabled.",
427       logKey.getScopes());
428   }
429 
430   @Test
431   public void testBulkLoadWALEdits() throws Exception {
432     // 1. Create wal key
433     WALKey logKey = new WALKey();
434     // 2. Get the bulk load wal edit event
435     WALEdit logEdit = getBulkLoadWALEdit();
436     // 3. Enable bulk load hfile replication
437     Configuration bulkLoadConf = HBaseConfiguration.create(conf);
438     bulkLoadConf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
439 
440     // 4. Get the scopes for the key
441     Replication.scopeWALEdits(htd, logKey, logEdit, bulkLoadConf, manager);
442 
443     NavigableMap<byte[], Integer> scopes = logKey.getScopes();
444     // Assert family with replication scope global is present in the key scopes
445     assertTrue("This family scope is set to global, should be part of replication key scopes.",
446       scopes.containsKey(f1));
447     // Assert family with replication scope local is not present in the key scopes
448     assertFalse("This family scope is set to local, should not be part of replication key scopes",
449       scopes.containsKey(f2));
450   }
451 
452   private WALEdit getBulkLoadWALEdit() {
453     // 1. Create store files for the families
454     Map<byte[], List<Path>> storeFiles = new HashMap<>(1);
455     Map<String, Long> storeFilesSize = new HashMap<>(1);
456     List<Path> p = new ArrayList<>(1);
457     Path hfilePath1 = new Path(Bytes.toString(f1));
458     p.add(hfilePath1);
459     try {
460       storeFilesSize.put(hfilePath1.getName(), fs.getFileStatus(hfilePath1).getLen());
461     } catch (IOException e) {
462       LOG.debug("Failed to calculate the size of hfile " + hfilePath1);
463       storeFilesSize.put(hfilePath1.getName(), 0L);
464     }
465     storeFiles.put(f1, p);
466 
467     p = new ArrayList<>(1);
468     Path hfilePath2 = new Path(Bytes.toString(f2));
469     p.add(hfilePath2);
470     try {
471       storeFilesSize.put(hfilePath2.getName(), fs.getFileStatus(hfilePath2).getLen());
472     } catch (IOException e) {
473       LOG.debug("Failed to calculate the size of hfile " + hfilePath2);
474       storeFilesSize.put(hfilePath2.getName(), 0L);
475     }
476     storeFiles.put(f2, p);
477 
478     // 2. Create bulk load descriptor
479     BulkLoadDescriptor desc = ProtobufUtil.toBulkLoadDescriptor(hri.getTable(),
480       ByteStringer.wrap(hri.getEncodedNameAsBytes()), storeFiles, storeFilesSize, 1);
481 
482     // 3. create bulk load wal edit event
483     WALEdit logEdit = WALEdit.createBulkLoadEvent(hri, desc);
484     return logEdit;
485   }
486 
487   static class DummyNodeFailoverWorker extends Thread {
488     private SortedMap<String, SortedSet<String>> logZnodesMap;
489     Server server;
490     private String deadRsZnode;
491     ReplicationQueues rq;
492 
493     public DummyNodeFailoverWorker(String znode, Server s) throws Exception {
494       this.deadRsZnode = znode;
495       this.server = s;
496       this.rq =
497           ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
498             server);
499       this.rq.init(this.server.getServerName().toString());
500     }
501 
502     @Override
503     public void run() {
504       try {
505         logZnodesMap = rq.claimQueues(deadRsZnode);
506         server.abort("Done with testing", null);
507       } catch (Exception e) {
508         LOG.error("Got exception while running NodeFailoverWorker", e);
509       } finally {
510         latch.countDown();
511       }
512     }
513 
514     /**
515      * @return 1 when the map is not empty.
516      */
517     private int isLogZnodesMapPopulated() {
518       Collection<SortedSet<String>> sets = logZnodesMap.values();
519       if (sets.size() > 1) {
520         throw new RuntimeException("unexpected size of logZnodesMap: " + sets.size());
521       }
522       if (sets.size() == 1) {
523         SortedSet<String> s = sets.iterator().next();
524         for (String file : files) {
525           // at least one file was missing
526           if (!s.contains(file)) {
527             return 0;
528           }
529         }
530         return 1; // we found all the files
531       }
532       return 0;
533     }
534   }
535 
536   static class DummyServer implements Server {
537     String hostname;
538 
539     DummyServer() {
540       hostname = "hostname.example.org";
541     }
542 
543     DummyServer(String hostname) {
544       this.hostname = hostname;
545     }
546 
547     @Override
548     public Configuration getConfiguration() {
549       return conf;
550     }
551 
552     @Override
553     public ZooKeeperWatcher getZooKeeper() {
554       return zkw;
555     }
556 
557     @Override
558     public CoordinatedStateManager getCoordinatedStateManager() {
559       return null;
560     }
561     @Override
562     public ClusterConnection getConnection() {
563       return null;
564     }
565 
566     @Override
567     public MetaTableLocator getMetaTableLocator() {
568       return null;
569     }
570 
571     @Override
572     public ServerName getServerName() {
573       return ServerName.valueOf(hostname, 1234, 1L);
574     }
575 
576     @Override
577     public void abort(String why, Throwable e) {
578       // To change body of implemented methods use File | Settings | File Templates.
579     }
580 
581     @Override
582     public boolean isAborted() {
583       return false;
584     }
585 
586     @Override
587     public void stop(String why) {
588       // To change body of implemented methods use File | Settings | File Templates.
589     }
590 
591     @Override
592     public boolean isStopped() {
593       return false; // To change body of implemented methods use File | Settings | File Templates.
594     }
595 
596     @Override
597     public ChoreService getChoreService() {
598       return null;
599     }
600   }
601 }