1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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.assertNotNull;
24 import static org.junit.Assert.assertNull;
25 import static org.junit.Assert.assertTrue;
26
27 import com.google.common.collect.Sets;
28 import java.io.IOException;
29 import java.lang.reflect.Field;
30 import java.net.URLEncoder;
31 import java.util.ArrayList;
32 import java.util.Collection;
33 import java.util.HashMap;
34 import java.util.List;
35 import java.util.Map;
36 import java.util.NavigableMap;
37 import java.util.Set;
38 import java.util.SortedMap;
39 import java.util.SortedSet;
40 import java.util.TreeMap;
41 import java.util.TreeSet;
42 import java.util.UUID;
43 import java.util.concurrent.CountDownLatch;
44
45 import org.apache.commons.logging.Log;
46 import org.apache.commons.logging.LogFactory;
47 import org.apache.hadoop.conf.Configuration;
48 import org.apache.hadoop.fs.FileSystem;
49 import org.apache.hadoop.fs.Path;
50 import org.apache.hadoop.hbase.HBaseConfiguration;
51 import org.apache.hadoop.hbase.HColumnDescriptor;
52 import org.apache.hadoop.hbase.HConstants;
53 import org.apache.hadoop.hbase.HTableDescriptor;
54 import org.apache.hadoop.hbase.KeyValue;
55 import org.apache.hadoop.hbase.Server;
56 import org.apache.hadoop.hbase.Stoppable;
57 import org.apache.hadoop.hbase.Waiter;
58 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
59 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.BulkLoadDescriptor;
60 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
61 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
62 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
63 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
64 import org.apache.hadoop.hbase.replication.ReplicationFactory;
65 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
66 import org.apache.hadoop.hbase.replication.ReplicationPeers;
67 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
68 import org.apache.hadoop.hbase.replication.ReplicationQueues;
69 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
70 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
71 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
72 import org.apache.hadoop.hbase.replication.regionserver.helper.DummyServer;
73 import org.apache.hadoop.hbase.testclassification.MediumTests;
74 import org.apache.hadoop.hbase.util.ByteStringer;
75 import org.apache.hadoop.hbase.util.Bytes;
76 import org.apache.hadoop.hbase.util.Pair;
77 import org.apache.hadoop.hbase.wal.WAL;
78 import org.apache.hadoop.hbase.wal.WALFactory;
79 import org.apache.hadoop.hbase.wal.WALKey;
80
81 import org.junit.Test;
82 import org.junit.experimental.categories.Category;
83
84 @Category(MediumTests.class)
85 public class TestReplicationSourceManagerManager extends TestReplicationSourceManagerBase {
86 private static final Log LOG =
87 LogFactory.getLog(TestReplicationSourceManagerManager.class);
88 private static List<String> files = new ArrayList<>();
89 private static CountDownLatch latch;
90
91 @Test
92 public void testLogRoll() throws Exception {
93 long baseline = 1000;
94 long time = baseline;
95 MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
96 KeyValue kv = new KeyValue(r1, f1, r1);
97 WALEdit edit = new WALEdit();
98 edit.add(kv);
99
100 List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
101 listeners.add(replication);
102 final WALFactory wals = new WALFactory(utility.getConfiguration(), listeners,
103 URLEncoder.encode("regionserver:60020", "UTF8"));
104 final WAL wal = wals.getWAL(hri.getEncodedNameAsBytes(), hri.getTable().getNamespace());
105 manager.init();
106 HTableDescriptor htd = new HTableDescriptor();
107 htd.addFamily(new HColumnDescriptor(f1));
108
109 for(long i = 1; i < 101; i++) {
110 if(i > 1 && i % 20 == 0) {
111 wal.rollWriter();
112 }
113 LOG.info(i);
114 final long txid = wal.append(htd,
115 hri,
116 new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc),
117 edit,
118 true);
119 wal.sync(txid);
120 }
121
122
123
124 LOG.info(baseline + " and " + time);
125 baseline += 101;
126 time = baseline;
127 LOG.info(baseline + " and " + time);
128
129 for (int i = 0; i < 3; i++) {
130 wal.append(htd, hri,
131 new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc),
132 edit,
133 true);
134 }
135 wal.sync();
136
137 int logNumber = 0;
138 for (Map.Entry<String, SortedSet<String>> entry : manager.getWALs().get(slaveId).entrySet()) {
139 logNumber += entry.getValue().size();
140 }
141 assertEquals(6, logNumber);
142
143 wal.rollWriter();
144
145 manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(),
146 "1", 0, false, false);
147
148 wal.append(htd, hri,
149 new WALKey(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc),
150 edit,
151 true);
152 wal.sync();
153
154 assertEquals(1, manager.getWALs().size());
155
156
157
158 }
159
160 @Test
161 public void testClaimQueues() throws Exception {
162 conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
163 final Server server = new DummyServer(conf, "hostname0.example.org", zkw);
164 ReplicationQueues rq =
165 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
166 server);
167 rq.init(server.getServerName().toString());
168
169 files.add("log1");
170 files.add("log2");
171 for (String file : files) {
172 rq.addLog("1", file);
173 }
174
175 Server s1 = new DummyServer(conf, "dummyserver1.example.org", zkw);
176 Server s2 = new DummyServer(conf, "dummyserver2.example.org", zkw);
177 Server s3 = new DummyServer(conf, "dummyserver3.example.org", zkw);
178
179
180 DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(
181 server.getServerName().getServerName(), s1);
182 DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(
183 server.getServerName().getServerName(), s2);
184 DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(
185 server.getServerName().getServerName(), s3);
186
187 latch = new CountDownLatch(3);
188
189 w1.start();
190 w2.start();
191 w3.start();
192
193 int populatedMap = 0;
194
195 latch.await();
196 populatedMap += w1.isLogZnodesMapPopulated() + w2.isLogZnodesMapPopulated()
197 + w3.isLogZnodesMapPopulated();
198 assertEquals(1, populatedMap);
199 server.abort("", null);
200 }
201
202 @Test
203 public void testCleanupFailoverQueues() throws Exception {
204 final Server server = new DummyServer(conf, "hostname1.example.org", zkw);
205 ReplicationQueues rq =
206 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
207 server);
208 rq.init(server.getServerName().toString());
209
210 SortedSet<String> files = new TreeSet<String>();
211 String group = "testgroup";
212 String file1 = group + ".log1";
213 String file2 = group + ".log2";
214 files.add(file1);
215 files.add(file2);
216 for (String file : files) {
217 rq.addLog("1", file);
218 }
219 Server s1 = new DummyServer(conf, "dummyserver1.example.org", zkw);
220 ReplicationQueues rq1 =
221 ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
222 rq1.init(s1.getServerName().toString());
223 ReplicationPeers rp1 =
224 ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration(), s1);
225 rp1.init();
226 NodeFailoverWorker w1 =
227 manager.new NodeFailoverWorker(server.getServerName().getServerName(), rq1, rp1, new UUID(
228 new Long(1), new Long(2)));
229 w1.start();
230 w1.join(10000);
231 assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
232 String id = "1-" + server.getServerName().getServerName();
233 assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id).get(group));
234 manager.cleanOldLogs(file2, id, true);
235
236 assertEquals(Sets.newHashSet(file2), manager.getWalsByIdRecoveredQueues().get(id).get(group));
237 }
238
239 @Test
240 public void testNodeFailoverDeadServerParsing() throws Exception {
241 LOG.debug("testNodeFailoverDeadServerParsing");
242 conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
243 final Server server = new DummyServer(conf, "ec2-54-234-230-108.compute-1.amazonaws.com", zkw);
244 ReplicationQueues repQueues =
245 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), conf, server);
246 repQueues.init(server.getServerName().toString());
247
248 files.add("log1");
249 files.add("log2");
250 for (String file : files) {
251 repQueues.addLog("1", file);
252 }
253
254
255 Server s1 = new DummyServer(conf, "ip-10-8-101-114.ec2.internal", zkw);
256 Server s2 = new DummyServer(conf, "ec2-107-20-52-47.compute-1.amazonaws.com", zkw);
257 Server s3 = new DummyServer(conf, "ec2-23-20-187-167.compute-1.amazonaws.com", zkw);
258
259
260 ReplicationQueues rq1 =
261 ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
262 rq1.init(s1.getServerName().toString());
263 String serverName = server.getServerName().getServerName();
264 List<String> unclaimed = rq1.getUnClaimedQueueIds(serverName);
265 rq1.claimQueue(serverName, unclaimed.get(0)).getSecond();
266 rq1.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
267
268 ReplicationQueues rq2 =
269 ReplicationFactory.getReplicationQueues(s2.getZooKeeper(), s2.getConfiguration(), s2);
270 rq2.init(s2.getServerName().toString());
271 serverName = s1.getServerName().getServerName();
272 unclaimed = rq2.getUnClaimedQueueIds(serverName);
273 rq2.claimQueue(serverName, unclaimed.get(0)).getSecond();
274 rq2.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
275 ReplicationQueues rq3 =
276 ReplicationFactory.getReplicationQueues(s3.getZooKeeper(), s3.getConfiguration(), s3);
277 rq3.init(s3.getServerName().toString());
278 serverName = s2.getServerName().getServerName();
279 unclaimed = rq3.getUnClaimedQueueIds(serverName);
280 String queue3 = rq3.claimQueue(serverName, unclaimed.get(0)).getFirst();
281 rq3.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
282 ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(queue3);
283 List<String> result = replicationQueueInfo.getDeadRegionServers();
284
285
286 assertTrue(result.contains(server.getServerName().getServerName()));
287 assertTrue(result.contains(s1.getServerName().getServerName()));
288 assertTrue(result.contains(s2.getServerName().getServerName()));
289
290 server.abort("", null);
291 }
292
293 @Test
294 public void testFailoverDeadServerCversionChange() throws Exception {
295 LOG.debug("testFailoverDeadServerCversionChange");
296
297 conf.setBoolean(HConstants.ZOOKEEPER_USEMULTI, true);
298 final Server s0 = new DummyServer(conf, "cversion-change0.example.org", zkw);
299 ReplicationQueues repQueues =
300 ReplicationFactory.getReplicationQueues(s0.getZooKeeper(), conf, s0);
301 repQueues.init(s0.getServerName().toString());
302
303 files.add("log1");
304 files.add("log2");
305 for (String file : files) {
306 repQueues.addLog("1", file);
307 }
308
309 Server s1 = new DummyServer(conf, "cversion-change1.example.org", zkw);
310 ReplicationQueues rq1 =
311 ReplicationFactory.getReplicationQueues(s1.getZooKeeper(), s1.getConfiguration(), s1);
312 rq1.init(s1.getServerName().toString());
313
314 ReplicationQueuesClient client =
315 ReplicationFactory.getReplicationQueuesClient(s1.getZooKeeper(), s1.getConfiguration(), s1);
316
317 int v0 = client.getQueuesZNodeCversion();
318 List<String> queues = rq1.getUnClaimedQueueIds(s0.getServerName().getServerName());
319 for(String queue : queues) {
320 rq1.claimQueue(s0.getServerName().getServerName(), queue);
321 }
322 rq1.removeReplicatorIfQueueIsEmpty(s0.getServerName().getServerName());
323 int v1 = client.getQueuesZNodeCversion();
324
325 assertEquals(v0 + 1, v1);
326
327 s0.abort("", null);
328 }
329
330 @Test
331 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RU_INVOKE_RUN",
332 justification="Intended")
333 public void testCleanupUnknownPeerZNode() throws Exception {
334 final Server server = new DummyServer(conf, "hostname2.example.org", zkw);
335 ReplicationQueues rq =
336 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
337 server);
338 rq.init(server.getServerName().toString());
339
340
341 String group = "testgroup";
342 rq.addLog("2", group + ".log1");
343 rq.addLog("2", group + ".log2");
344
345 NodeFailoverWorker w1 = manager.new NodeFailoverWorker(server.getServerName().getServerName());
346 w1.run();
347
348
349 for (String peer : manager.getAllQueues()) {
350 assertTrue(peer.startsWith("1"));
351 }
352 }
353
354 @Test
355 public void testBulkLoadWALEditsWithoutBulkLoadReplicationEnabled() throws Exception {
356
357 WALKey logKey = new WALKey();
358
359 WALEdit logEdit = getBulkLoadWALEdit();
360
361
362 Replication.scopeWALEdits(htd, logKey, logEdit, conf, manager);
363
364
365 assertNull("No bulk load entries scope should be added if bulk load replication is diabled.",
366 logKey.getScopes());
367 }
368
369 @Test
370 public void testBulkLoadWALEdits() throws Exception {
371
372 WALKey logKey = new WALKey();
373
374 WALEdit logEdit = getBulkLoadWALEdit();
375
376 Configuration bulkLoadConf = HBaseConfiguration.create(conf);
377 bulkLoadConf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
378
379
380 Replication.scopeWALEdits(htd, logKey, logEdit, bulkLoadConf, manager);
381
382 NavigableMap<byte[], Integer> scopes = logKey.getScopes();
383
384 assertTrue("This family scope is set to global, should be part of replication key scopes.",
385 scopes.containsKey(f1));
386
387 assertFalse("This family scope is set to local, should not be part of replication key scopes",
388 scopes.containsKey(f2));
389 }
390
391
392
393
394
395
396
397
398 @Test
399 public void testPeerRemovalCleanup() throws Exception {
400 String replicationSourceImplName = conf.get("replication.replicationsource.implementation");
401 final String peerId = "FakePeer";
402 final ReplicationPeerConfig peerConfig =
403 new ReplicationPeerConfig().setClusterKey("localhost:1:/hbase");
404 try {
405 final ReplicationQueues rq =
406 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
407 server);
408 rq.init(server.getServerName().toString());
409
410
411 conf.set("replication.replicationsource.implementation",
412 FailInitializeDummyReplicationSource.class.getName());
413 final ReplicationPeers rp = manager.getReplicationPeers();
414
415
416 addPeerAndWait(peerId, peerConfig, false);
417
418
419 assertNull(manager.getSource(peerId));
420
421
422
423 rq.addLog(peerId, "FakeFile");
424
425
426 removePeerAndWait(peerId);
427 assertFalse(rq.getAllQueues().contains(peerId));
428 } finally {
429 conf.set("replication.replicationsource.implementation", replicationSourceImplName);
430 removePeerAndWait(peerId);
431 }
432 }
433
434 @Test
435 public void testRemovePeerMetricsCleanup() throws Exception {
436 final String peerId = "DummyPeer";
437 final ReplicationPeerConfig peerConfig =
438 new ReplicationPeerConfig().setClusterKey("localhost:1:/hbase");
439 try {
440 addPeerAndWait(peerId, peerConfig, true);
441
442 ReplicationSourceInterface source = manager.getSource(peerId);
443
444 assertNotNull(source);
445
446 Field f = MetricsSource.class.getDeclaredField("globalSourceSource");
447 f.setAccessible(true);
448 MetricsReplicationSourceSource globalSource =
449 (MetricsReplicationSourceSource)f.get(source.getSourceMetrics());
450 int globalLogQueueSizeInitial = globalSource.getSizeOfLogQueue();
451
452
453 source.enqueueLog(new Path("abc"));
454 assertEquals(1, source.getSourceMetrics().getSizeOfLogQueue());
455 assertEquals(1 + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
456
457
458 removePeerAndWait(peerId);
459 assertEquals(globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
460
461
462 addPeerAndWait(peerId, peerConfig, true);
463 source = manager.getSource(peerId);
464 assertNotNull(source);
465 assertEquals(0, source.getSourceMetrics().getSizeOfLogQueue());
466 assertEquals(globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
467 } finally {
468 removePeerAndWait(peerId);
469 }
470 }
471
472
473
474
475
476
477
478
479 private void addPeerAndWait(final String peerId, final ReplicationPeerConfig peerConfig,
480 final boolean waitForSource) throws Exception {
481 final ReplicationPeers rp = manager.getReplicationPeers();
482 rp.addPeer(peerId, peerConfig);
483 Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
484 @Override public boolean evaluate() throws Exception {
485 if (waitForSource) {
486 return (manager.getSource(peerId) != null);
487 } else {
488 return (rp.getPeer(peerId) != null);
489 }
490 }
491 });
492 }
493
494
495
496
497
498
499 private void removePeerAndWait(final String peerId) throws Exception {
500 final ReplicationPeers rp = manager.getReplicationPeers();
501 if (rp.getAllPeerIds().contains(peerId)) {
502 rp.removePeer(peerId);
503 }
504 Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
505 @Override public boolean evaluate() throws Exception {
506 List<String> peers = rp.getAllPeerIds();
507 return (!manager.getAllQueues().contains(peerId)) && (rp.getPeer(peerId) == null)
508 && (!peers.contains(peerId));
509 }
510 });
511 }
512
513 @Test
514 public void testSameWALPrefix() throws IOException {
515 Set<Path> latestWalsBefore = manager.getLastestPath();
516 Path walName1 = new Path("localhost,8080,12345-45678-Peer.34567");
517 Path walName2 = new Path("localhost,8080,12345.56789");
518 manager.preLogRoll(walName1);
519 manager.preLogRoll(walName2);
520 Set<Path> latestWals = manager.getLastestPath();
521 latestWals.removeAll(latestWalsBefore);
522 assertEquals(2, latestWals.size());
523 assertTrue(latestWals.contains(walName1));
524 assertTrue(latestWals.contains(walName2));
525 }
526
527 private WALEdit getBulkLoadWALEdit() {
528
529 Map<byte[], List<Path>> storeFiles = new HashMap<>(1);
530 Map<String, Long> storeFilesSize = new HashMap<>(1);
531 List<Path> p = new ArrayList<>(1);
532 Path hfilePath1 = new Path(Bytes.toString(f1));
533 p.add(hfilePath1);
534 try {
535 storeFilesSize.put(hfilePath1.getName(), fs.getFileStatus(hfilePath1).getLen());
536 } catch (IOException e) {
537 LOG.debug("Failed to calculate the size of hfile " + hfilePath1);
538 storeFilesSize.put(hfilePath1.getName(), 0L);
539 }
540 storeFiles.put(f1, p);
541
542 p = new ArrayList<>(1);
543 Path hfilePath2 = new Path(Bytes.toString(f2));
544 p.add(hfilePath2);
545 try {
546 storeFilesSize.put(hfilePath2.getName(), fs.getFileStatus(hfilePath2).getLen());
547 } catch (IOException e) {
548 LOG.debug("Failed to calculate the size of hfile " + hfilePath2);
549 storeFilesSize.put(hfilePath2.getName(), 0L);
550 }
551 storeFiles.put(f2, p);
552
553
554 BulkLoadDescriptor desc =
555 ProtobufUtil.toBulkLoadDescriptor(hri.getTable(),
556 ByteStringer.wrap(hri.getEncodedNameAsBytes()), storeFiles, storeFilesSize, 1);
557
558
559 WALEdit logEdit = WALEdit.createBulkLoadEvent(hri, desc);
560 return logEdit;
561 }
562
563 static class DummyNodeFailoverWorker extends Thread {
564 private SortedMap<String, SortedSet<String>> logZnodesMap;
565 Server server;
566 private String deadRsZnode;
567 ReplicationQueues rq;
568
569 public DummyNodeFailoverWorker(String znode, Server s) throws Exception {
570 this.deadRsZnode = znode;
571 this.server = s;
572 this.rq =
573 ReplicationFactory.getReplicationQueues(server.getZooKeeper(), server.getConfiguration(),
574 server);
575 this.rq.init(this.server.getServerName().toString());
576 }
577
578 @Override
579 public void run() {
580 try {
581 logZnodesMap = new TreeMap<>();
582 List<String> queues = rq.getUnClaimedQueueIds(deadRsZnode);
583 for(String queue:queues){
584 Pair<String, SortedSet<String>> pair = rq.claimQueue(deadRsZnode, queue);
585 if (pair != null) {
586 logZnodesMap.put(pair.getFirst(), pair.getSecond());
587 }
588 }
589 server.abort("Done with testing", null);
590 } catch (Exception e) {
591 LOG.error("Got exception while running NodeFailoverWorker", e);
592 } finally {
593 latch.countDown();
594 }
595 }
596
597
598
599
600 private int isLogZnodesMapPopulated() {
601 Collection<SortedSet<String>> sets = logZnodesMap.values();
602 if (sets.size() > 1) {
603 throw new RuntimeException("unexpected size of logZnodesMap: " + sets.size());
604 }
605 if (sets.size() == 1) {
606 SortedSet<String> s = sets.iterator().next();
607 for (String file : files) {
608
609 if (!s.contains(file)) {
610 return 0;
611 }
612 }
613 return 1;
614 }
615 return 0;
616 }
617 }
618
619 static class FailInitializeDummyReplicationSource extends ReplicationSourceDummy {
620
621 @Override
622 public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
623 ReplicationQueues rq, ReplicationPeers rp, Stoppable stopper, String peerClusterId,
624 UUID clusterId, ReplicationEndpoint replicationEndpoint, MetricsSource metrics)
625 throws IOException {
626 throw new IOException("Failing deliberately");
627 }
628 }
629 }