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