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  
20  package org.apache.hadoop.hbase.replication;
21  
22  import static org.apache.hadoop.hbase.replication.TestReplicationEndpoint.ReplicationEndpointForTest;
23  import static org.hamcrest.CoreMatchers.is;
24  import static org.hamcrest.MatcherAssert.assertThat;
25  import static org.junit.Assert.assertEquals;
26  import static org.junit.Assert.assertNotNull;
27  import static org.junit.Assert.assertNull;
28  import static org.junit.Assert.assertTrue;
29  import static org.mockito.Matchers.anyBoolean;
30  import static org.mockito.Matchers.anyString;
31  import static org.mockito.Mockito.doCallRealMethod;
32  import static org.mockito.Mockito.doReturn;
33  import static org.mockito.Mockito.doThrow;
34  import static org.mockito.Mockito.mock;
35  import static org.mockito.Mockito.verify;
36  import static org.mockito.Mockito.when;
37  import static org.mockito.internal.verification.VerificationModeFactory.times;
38  import com.google.common.collect.Lists;
39  import com.google.common.collect.Sets;
40  import java.io.IOException;
41  import java.util.Collections;
42  import java.util.HashMap;
43  import java.util.List;
44  import java.util.Map;
45  import java.util.NavigableMap;
46  import java.util.SortedSet;
47  import java.util.TreeMap;
48  import java.util.UUID;
49  import java.util.concurrent.ExecutorService;
50  import java.util.concurrent.Executors;
51  import java.util.concurrent.Future;
52  import java.util.concurrent.atomic.AtomicLong;
53  import org.apache.commons.logging.Log;
54  import org.apache.commons.logging.LogFactory;
55  import org.apache.hadoop.conf.Configuration;
56  import org.apache.hadoop.fs.FSDataOutputStream;
57  import org.apache.hadoop.fs.FileSystem;
58  import org.apache.hadoop.fs.Path;
59  import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
60  import org.apache.hadoop.hbase.CoordinatedStateManager;
61  import org.apache.hadoop.hbase.HBaseConfiguration;
62  import org.apache.hadoop.hbase.HBaseTestingUtility;
63  import org.apache.hadoop.hbase.HConstants;
64  import org.apache.hadoop.hbase.HRegionInfo;
65  import org.apache.hadoop.hbase.KeyValue;
66  import org.apache.hadoop.hbase.MiniHBaseCluster;
67  import org.apache.hadoop.hbase.Server;
68  import org.apache.hadoop.hbase.Stoppable;
69  import org.apache.hadoop.hbase.TableName;
70  import org.apache.hadoop.hbase.Waiter;
71  import org.apache.hadoop.hbase.Waiter.Predicate;
72  import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
73  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
74  import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
75  import org.apache.hadoop.hbase.regionserver.HRegionServer;
76  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
77  import org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint;
78  import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceFactory;
79  import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceSource;
80  import org.apache.hadoop.hbase.replication.regionserver.MetricsSource;
81  import org.apache.hadoop.hbase.replication.regionserver.Replication;
82  import org.apache.hadoop.hbase.replication.regionserver.ReplicationSource;
83  import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface;
84  import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
85  import org.apache.hadoop.hbase.testclassification.MediumTests;
86  import org.apache.hadoop.hbase.util.ByteStringer;
87  import org.apache.hadoop.hbase.util.Bytes;
88  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
89  import org.apache.hadoop.hbase.util.HFileTestUtil;
90  import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
91  import org.apache.hadoop.hbase.wal.WAL;
92  import org.apache.hadoop.hbase.wal.WALFactory;
93  import org.apache.hadoop.hbase.wal.WALKey;
94  import org.apache.hadoop.hbase.wal.WALProvider;
95  
96  import org.junit.After;
97  import org.junit.AfterClass;
98  import org.junit.Before;
99  import org.junit.BeforeClass;
100 import org.junit.Test;
101 import org.junit.experimental.categories.Category;
102 import org.mockito.ArgumentCaptor;
103 import org.mockito.Mockito;
104 
105 @Category(MediumTests.class)
106 public class TestReplicationSource {
107 
108   private static final Log LOG =
109       LogFactory.getLog(TestReplicationSource.class);
110   private final static HBaseTestingUtility TEST_UTIL =
111       new HBaseTestingUtility();
112   private final static HBaseTestingUtility TEST_UTIL_PEER =
113       new HBaseTestingUtility();
114   private static FileSystem FS;
115   private static Path oldLogDir;
116   private static Path logDir;
117   private static Configuration conf = TEST_UTIL.getConfiguration();
118 
119   /**
120    * @throws java.lang.Exception exception
121    */
122   @BeforeClass
123   public static void setUpBeforeClass() throws Exception {
124     TEST_UTIL.startMiniDFSCluster(1);
125     FS = TEST_UTIL.getDFSCluster().getFileSystem();
126     Path rootDir = TEST_UTIL.createRootDir();
127     oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
128     if (FS.exists(oldLogDir)) {
129       FS.delete(oldLogDir, true);
130     }
131     logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
132     if (FS.exists(logDir)) {
133       FS.delete(logDir, true);
134     }
135     conf.setBoolean("replication.source.eof.autorecovery", true);
136   }
137 
138   @Before
139   public void setup() throws IOException {
140     if (!FS.exists(logDir)) {
141       FS.mkdirs(logDir);
142     }
143     if (!FS.exists(oldLogDir)) {
144       FS.mkdirs(oldLogDir);
145     }
146 
147     ReplicationEndpointForTest.contructedCount.set(0);
148     ReplicationEndpointForTest.startedCount.set(0);
149     ReplicationEndpointForTest.replicateCount.set(0);
150     ReplicationEndpointForTest.stoppedCount.set(0);
151     ReplicationEndpointForTest.lastEntries = null;
152   }
153 
154   @After
155   public void tearDown() throws IOException {
156     if (FS.exists(oldLogDir)) {
157       FS.delete(oldLogDir, true);
158     }
159     if (FS.exists(logDir)) {
160       FS.delete(logDir, true);
161     }
162   }
163 
164   @AfterClass
165   public static void tearDownAfterClass() throws Exception {
166     TEST_UTIL_PEER.shutdownMiniHBaseCluster();
167     TEST_UTIL.shutdownMiniHBaseCluster();
168     TEST_UTIL.shutdownMiniDFSCluster();
169   }
170 
171   /**
172    * Sanity check that we can move logs around while we are reading
173    * from them. Should this test fail, ReplicationSource would have a hard
174    * time reading logs that are being archived.
175    * @throws Exception exception
176    */
177   @Test
178   public void testLogMoving() throws Exception{
179     Path logPath = new Path(logDir, "log");
180     WALProvider.Writer writer = WALFactory.createWALWriter(FS, logPath,
181         TEST_UTIL.getConfiguration());
182     for(int i = 0; i < 3; i++) {
183       byte[] b = Bytes.toBytes(Integer.toString(i));
184       KeyValue kv = new KeyValue(b, b, b);
185       WALEdit edit = new WALEdit();
186       edit.add(kv);
187       WALKey key = new WALKey(b, TableName.valueOf(b), 0, 0,
188           HConstants.DEFAULT_CLUSTER_ID);
189       writer.append(new WAL.Entry(key, edit));
190       writer.sync(false);
191     }
192     writer.close();
193 
194     WAL.Reader reader = WALFactory.createReader(FS, logPath, TEST_UTIL.getConfiguration());
195     WAL.Entry entry = reader.next();
196     assertNotNull(entry);
197 
198     Path oldLogPath = new Path(oldLogDir, "log");
199     FS.rename(logPath, oldLogPath);
200 
201     entry = reader.next();
202     assertNotNull(entry);
203 
204     entry = reader.next();
205     entry = reader.next();
206 
207     assertNull(entry);
208     reader.close();
209   }
210 
211   /**
212    * Tests that {@link ReplicationSource#terminate(String)} will timeout properly
213    */
214   @Test
215   public void testTerminateTimeout() throws Exception {
216     final ReplicationSource source = new ReplicationSource();
217     ReplicationEndpoint replicationEndpoint = new HBaseInterClusterReplicationEndpoint() {
218       @Override
219       protected void doStart() {
220         notifyStarted();
221       }
222 
223       @Override
224       protected void doStop() {
225         // not calling notifyStopped() here causes the caller of stop() to get a Future that never
226         // completes
227       }
228     };
229     replicationEndpoint.start();
230     ReplicationPeers mockPeers = Mockito.mock(ReplicationPeers.class);
231     ReplicationPeer mockPeer = Mockito.mock(ReplicationPeer.class);
232     Mockito.when(mockPeer.getPeerBandwidth()).thenReturn(0L);
233     Configuration testConf = HBaseConfiguration.create();
234     testConf.setInt("replication.source.maxretriesmultiplier", 1);
235     ReplicationSourceManager manager = Mockito.mock(ReplicationSourceManager.class);
236     source.init(testConf, null, manager, null, mockPeers, null, "testPeer",
237         null, replicationEndpoint, null);
238     ExecutorService executor = Executors.newSingleThreadExecutor();
239     final Future<?> future = executor.submit(new Runnable() {
240 
241       @Override
242       public void run() {
243         source.terminate("testing source termination");
244       }
245     });
246     long sleepForRetries = testConf.getLong("replication.source.sleepforretries", 1000);
247     Waiter.waitFor(testConf, sleepForRetries * 2, new Predicate<Exception>() {
248 
249       @Override
250       public boolean evaluate() throws Exception {
251         return future.isDone();
252       }
253     });
254   }
255 
256   private void appendEntries(WALProvider.Writer writer, int numEntries) throws IOException {
257     for (int i = 0; i < numEntries; i++) {
258       byte[] b = Bytes.toBytes(Integer.toString(i));
259       KeyValue kv = new KeyValue(b, b, b);
260       WALEdit edit = new WALEdit();
261       edit.add(kv);
262       WALKey key = new WALKey(b, TableName.valueOf(b), 0, 0, HConstants.DEFAULT_CLUSTER_ID);
263       NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
264       scopes.put(b, HConstants.REPLICATION_SCOPE_GLOBAL);
265       key.setScopes(scopes);
266       writer.append(new WAL.Entry(key, edit));
267       writer.sync(false);
268     }
269     writer.close();
270   }
271 
272   private long getPosition(WALFactory wals, Path log2, int numEntries) throws IOException {
273     WAL.Reader reader = wals.createReader(FS, log2);
274     for (int i = 0; i < numEntries; i++) {
275       reader.next();
276     }
277     return reader.getPosition();
278   }
279 
280   private static final class Mocks {
281     private ReplicationSourceManager manager = mock(ReplicationSourceManager.class);
282     private final ReplicationQueues queues = mock(ReplicationQueues.class);
283     private final ReplicationPeers peers = mock(ReplicationPeers.class);
284     private final MetricsSource metrics = mock(MetricsSource.class);
285     private final ReplicationPeer peer = mock(ReplicationPeer.class);
286     private final ReplicationEndpoint.Context context = mock(ReplicationEndpoint.Context.class);
287     private final AtomicLong totalBufferUsed = new AtomicLong();
288 
289     private Mocks() {
290       when(peers.getStatusOfPeer(anyString())).thenReturn(true);
291       when(context.getReplicationPeer()).thenReturn(peer);
292       when(manager.getTotalBufferUsed()).thenReturn(totalBufferUsed);
293     }
294 
295     ReplicationSource createReplicationSourceAndManagerWithMocks(ReplicationEndpoint endpoint)
296         throws Exception {
297       ReplicationTracker tracker = mock(ReplicationTracker.class);
298       Server server = mock(Server.class);
299       FileSystem fs = mock(FileSystem.class);
300       UUID clusterId = UUID.randomUUID();
301       String peerId = "testPeerClusterZnode";
302 
303       manager = Mockito.spy(new ReplicationSourceManager(
304         queues, peers, tracker, conf, server, fs, logDir, oldLogDir, clusterId));
305 
306       doCallRealMethod().when(manager).removePeer(Mockito.anyString());
307       // Mock the failure during cleaning log with node already deleted
308       doThrow(new ReplicationSourceWithoutPeerException("Peer Removed")).when(queues)
309         .removeLog(anyString(), anyString());
310       doCallRealMethod().when(manager)
311         .logPositionAndCleanOldLogs(Mockito.<Path>anyObject(), Mockito.anyString(),
312           Mockito.anyLong(), Mockito.anyBoolean(), Mockito.anyBoolean());
313       final ReplicationSource source = new ReplicationSource();
314       endpoint.init(context);
315       source.init(conf, FS, manager, queues, peers, mock(Stoppable.class),
316         peerId, clusterId, endpoint, metrics);
317       manager.getSources().add(source);
318       SortedSet<String> walsWithPrefix = Sets.newTreeSet(Collections.singletonList("fake"));
319       doReturn(walsWithPrefix).when(manager).getLogsWithPrefix(anyString(), anyString());
320       return source;
321     }
322 
323     ReplicationSource createReplicationSourceWithMocks(ReplicationEndpoint endpoint,
324       boolean isRecovered) throws IOException {
325       final ReplicationSource source = new ReplicationSource();
326       endpoint.init(context);
327       source.init(conf, FS, manager, queues, peers, mock(Stoppable.class),
328         "testPeerClusterZnode", UUID.randomUUID(), endpoint, metrics);
329       if (isRecovered) {
330         when(manager.getOldSources())
331           .thenReturn(Lists.<ReplicationSourceInterface>newArrayList(source));
332       }
333       return source;
334     }
335 
336     ReplicationSource createReplicationSourceWithMocks(MetricsSource metrics,
337             ReplicationEndpoint endpoint) throws IOException {
338       final ReplicationSource source = new ReplicationSource();
339       endpoint.init(context);
340       source.init(conf, FS, manager, queues, peers, mock(Stoppable.class),
341         "testPeerClusterZnode", UUID.randomUUID(), endpoint, metrics);
342       return source;
343     }
344 
345     public AtomicLong getTotalBufferUsed() {
346       return totalBufferUsed;
347     }
348   }
349 
350   @Test
351   public void testSetLogPositionForWALCurrentlyReadingWhenLogsRolled() throws Exception {
352     final int numWALEntries = 5;
353     int nbCapacity = conf.getInt("replication.source.nb.capacity", 25000);
354     try {
355       conf.setInt("replication.source.nb.capacity", numWALEntries);
356 
357       Mocks mocks = new Mocks();
358       final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest() {
359         @Override public WALEntryFilter getWALEntryfilter() {
360           return null;
361         }
362       };
363       WALFactory wals = new WALFactory(TEST_UTIL.getConfiguration(), null, "test");
364       final Path log1 = new Path(logDir, "log.1");
365       final Path log2 = new Path(logDir, "log.2");
366 
367       WALProvider.Writer writer1
368         = WALFactory.createWALWriter(FS, log1, TEST_UTIL.getConfiguration());
369       WALProvider.Writer writer2
370         = WALFactory.createWALWriter(FS, log2, TEST_UTIL.getConfiguration());
371 
372       appendEntries(writer1, 3);
373       appendEntries(writer2, 2);
374 
375       long pos = getPosition(wals, log2, 2);
376 
377       final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint, false);
378       source.run();
379 
380       source.enqueueLog(log1);
381       // log rolled
382       source.enqueueLog(log2);
383 
384       Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
385         @Override public boolean evaluate() {
386           return endpoint.replicateCount.get() > 0;
387         }
388       });
389 
390       ArgumentCaptor<Path> pathCaptor = ArgumentCaptor.forClass(Path.class);
391       ArgumentCaptor<Long> positionCaptor = ArgumentCaptor.forClass(Long.class);
392       verify(mocks.manager, times(1))
393         .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(),
394           anyBoolean(), anyBoolean());
395       assertTrue(endpoint.lastEntries.size() == 5);
396       assertThat(pathCaptor.getValue(), is(log2));
397       assertThat(positionCaptor.getValue(), is(pos));
398     } finally {
399       conf.setInt("replication.source.nb.capacity", nbCapacity);
400     }
401   }
402 
403   @Test
404   public void testUpdateQuotaWhenBulkLoad() throws Exception {
405     byte[] cfBytes = Bytes.toBytes("cf");
406     TableName tableName = TableName.valueOf("test_table");
407     Path dir = TEST_UTIL.getDataTestDirOnTestFS(tableName.getNameAsString());
408     Map<String, Long> storeFilesSize = new HashMap<>(1);
409     Map<byte[], List<Path>> storeFiles = new HashMap<>(1);
410     int numRows = 10;
411 
412     Path familyDir = new Path(dir, Bytes.toString(cfBytes));
413     Path hfilePath = new Path(familyDir, "test_hfile");
414     HFileTestUtil.createHFile(conf, FS, hfilePath, cfBytes, cfBytes,
415       Bytes.toBytes("a"), Bytes.toBytes("z"), numRows);
416     storeFilesSize.put(hfilePath.getName(), FS.getFileStatus(hfilePath).getLen());
417     storeFiles.put(cfBytes, Collections.singletonList(hfilePath));
418 
419     HRegionInfo regionInfo = new HRegionInfo(tableName);
420     WALProtos.BulkLoadDescriptor loadDescriptor = ProtobufUtil
421       .toBulkLoadDescriptor(tableName, ByteStringer.wrap(Bytes.toBytes("test_region")),
422         storeFiles, storeFilesSize, 1, null);
423     WALEdit edit = WALEdit.createBulkLoadEvent(regionInfo, loadDescriptor);
424 
425     final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest() {
426       @Override
427       public WALEntryFilter getWALEntryfilter() {
428         return null;
429       }
430     };
431     final Path log = new Path(logDir, "log.1");
432 
433     WALProvider.Writer writer = WALFactory.createWALWriter(FS, log, TEST_UTIL.getConfiguration());
434     WALKey key = new WALKey(regionInfo.getEncodedNameAsBytes(), tableName, 0, 0,
435       HConstants.DEFAULT_CLUSTER_ID);
436     WAL.Entry bulkLoadEventEntry = new WAL.Entry(key, edit);
437     WAL.Entry entryWithoutCells = new WAL.Entry(key, new WALEdit());
438     writer.append(bulkLoadEventEntry);
439     writer.append(entryWithoutCells);
440     writer.close();
441 
442     Mocks mocks = new Mocks();
443     final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint, false);
444     source.run();
445 
446     source.enqueueLog(log);
447     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
448       @Override public boolean evaluate() throws Exception {
449         return endpoint.replicateCount.get() > 0;
450       }
451     });
452 
453     assertEquals(0L, mocks.getTotalBufferUsed().get());
454   }
455 
456   @Test
457   public void testSetLogPositionAndRemoveOldWALsEvenIfEmptyWALsRolled() throws Exception {
458     Mocks mocks = new Mocks();
459 
460     final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest();
461     final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint, false);
462     WALFactory wals = new WALFactory(TEST_UTIL.getConfiguration(), null, "test");
463 
464     final Path log1 = new Path(logDir, "log.1");
465     final Path log2 = new Path(logDir, "log.2");
466 
467     WALFactory.createWALWriter(FS, log1, TEST_UTIL.getConfiguration()).close();
468     WALFactory.createWALWriter(FS, log2, TEST_UTIL.getConfiguration()).close();
469     final long startPos = getPosition(wals, log2, 0);
470 
471     source.run();
472     source.enqueueLog(log1);
473     source.enqueueLog(log2);
474 
475     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
476       @Override public boolean evaluate() throws Exception {
477         return log2.equals(source.getLastLoggedPath())
478                 && source.getLastLoggedPosition() >= startPos;
479       }
480     });
481 
482     ArgumentCaptor<Path> pathCaptor = ArgumentCaptor.forClass(Path.class);
483     ArgumentCaptor<Long> positionCaptor = ArgumentCaptor.forClass(Long.class);
484 
485     verify(mocks.manager, times(1))
486             .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(),
487                     anyBoolean(), anyBoolean());
488     assertThat(pathCaptor.getValue(), is(log2));
489     assertThat(positionCaptor.getValue(), is(startPos));
490   }
491 
492   @Test
493   public void testSetLogPositionAndRemoveOldWALsEvenIfNoCfsReplicated() throws Exception {
494     Mocks mocks = new Mocks();
495     // set table cfs to filter all cells out
496     final TableName replicatedTable = TableName.valueOf("replicated_table");
497     final Map<TableName, List<String>> cfs =
498             Collections.singletonMap(replicatedTable, Collections.<String>emptyList());
499     when(mocks.peer.getTableCFs()).thenReturn(cfs);
500 
501     WALFactory wals = new WALFactory(TEST_UTIL.getConfiguration(), null, "test");
502     final Path log1 = new Path(logDir, "log.1");
503     final Path log2 = new Path(logDir, "log.2");
504 
505     WALProvider.Writer writer1 = WALFactory.createWALWriter(FS, log1, TEST_UTIL.getConfiguration());
506     WALProvider.Writer writer2 = WALFactory.createWALWriter(FS, log2, TEST_UTIL.getConfiguration());
507 
508     appendEntries(writer1, 3);
509     appendEntries(writer2, 2);
510     final long pos = getPosition(wals, log2, 2);
511 
512     final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest();
513     final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint, false);
514     source.enqueueLog(log1);
515     source.enqueueLog(log2);
516     source.run();
517     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
518       @Override public boolean evaluate() throws Exception {
519         // wait until reader read all cells
520         return log2.equals(source.getLastLoggedPath()) && source.getLastLoggedPosition() >= pos;
521       }
522     });
523 
524     ArgumentCaptor<Path> pathCaptor = ArgumentCaptor.forClass(Path.class);
525     ArgumentCaptor<Long> positionCaptor = ArgumentCaptor.forClass(Long.class);
526 
527     // all old wals should be removed by updating wal position, even if all cells are filtered out.
528     verify(mocks.manager, times(1))
529         .logPositionAndCleanOldLogs(pathCaptor.capture(), anyString(), positionCaptor.capture(),
530               anyBoolean(), anyBoolean());
531     assertThat(pathCaptor.getValue(), is(log2));
532     assertThat(positionCaptor.getValue(), is(pos));
533   }
534 
535   /**
536    * There can be a scenario of replication peer removed but the replication source
537    * still running since termination of source depends upon zk listener and there
538    * can a rare scenario where zk listener might not get invoked or get delayed.
539    * In that case, replication source manager will throw since it won't be able
540    * to remove the znode while removing the log. We should terminate the source
541    * in that case. See HBASE-25583
542    * @throws Exception any exception
543    */
544   @Test
545   public void testReplicationSourceTerminationWhenNoZnodeForPeerAndQueues() throws Exception {
546     final Mocks mocks = new Mocks();
547     // set table cfs to filter all cells out
548     final TableName replicatedTable = TableName.valueOf("replicated_table");
549     final Map<TableName, List<String>> cfs =
550       Collections.singletonMap(replicatedTable, Collections.<String>emptyList());
551     when(mocks.peer.getTableCFs()).thenReturn(cfs);
552 
553     // Append 3 entries in a log
554     final Path log1 = new Path(logDir, "log.1");
555     WALProvider.Writer writer1 = WALFactory.createWALWriter(FS, log1, TEST_UTIL.getConfiguration());
556     appendEntries(writer1, 3);
557 
558     // Replication end point with no filter
559     final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest() {
560       @Override
561       public WALEntryFilter getWALEntryfilter() {
562         return null;
563       }
564     };
565 
566     final ReplicationSource source = mocks.createReplicationSourceAndManagerWithMocks(endpoint);
567     source.startup();
568     // source thread should be active
569     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
570       @Override public boolean evaluate() {
571         return source.isAlive();
572       }
573     });
574     source.enqueueLog(log1);
575 
576     // Wait for source to replicate
577     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
578       @Override public boolean evaluate() {
579         return endpoint.replicateCount.get() == 1;
580       }
581     });
582 
583     // Wait for all the entries to get replicated
584     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
585       @Override public boolean evaluate() {
586         return endpoint.lastEntries.size() == 3;
587       }
588     });
589 
590     // And the source should be terminated
591     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
592       @Override public boolean evaluate() {
593         return !source.isSourceActive();
594       }
595     });
596 
597     // And the source thread be terminated
598     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
599       @Override public boolean evaluate() {
600         return !source.isAlive();
601       }
602     });
603     assertTrue("Source should be removed", mocks.manager.getSources().isEmpty());
604   }
605 
606   @Test
607   public void testReplicationOnEmptyLogAtTheEndOfQueueWithMultipleLogs() throws Exception {
608     final String logPrefix = "logPrefix";
609     Mocks mocks = new Mocks();
610     // set table cfs to filter all cells out
611     final TableName replicatedTable = TableName.valueOf("replicated_table");
612     final Map<TableName, List<String>> cfs =
613       Collections.singletonMap(replicatedTable, Collections.<String>emptyList());
614     when(mocks.peer.getTableCFs()).thenReturn(cfs);
615 
616     // Append 3 entries in a log
617     final Path log1 = new Path(logDir, logPrefix + ".1");
618     WALProvider.Writer writer1 = WALFactory.createWALWriter(FS, log1, TEST_UTIL.getConfiguration());
619     appendEntries(writer1, 3);
620 
621     // Create a 0 length log.
622     Path emptyLog = new Path(logDir, logPrefix + ".2");
623     FSDataOutputStream fsdos = FS.create(emptyLog);
624     fsdos.close();
625     assertEquals(0, FS.getFileStatus(emptyLog).getLen());
626 
627     // Replication end point with no filter
628     final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest() {
629       @Override
630       public WALEntryFilter getWALEntryfilter() {
631         return null;
632       }
633     };
634 
635     final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint, true);
636     source.run();
637     source.enqueueLog(log1);
638     source.enqueueLog(emptyLog);
639 
640     // Wait for source to replicate
641     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
642       @Override public boolean evaluate() {
643         return endpoint.replicateCount.get() == 1;
644       }
645     });
646 
647     // Wait and verify if all the entries get replicated for non empty logs
648     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
649       @Override public boolean evaluate() {
650         return endpoint.lastEntries.size() == 3;
651       }
652     });
653 
654     // Wait and verify if log queue has been drained fully
655     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
656       @Override public boolean evaluate() {
657         return source.getQueues().get(logPrefix).isEmpty();
658       }
659     });
660   }
661 
662   @Test
663   public void testReplicationOnEmptyLogAtTheEndOfQueueWithSingleLog() throws Exception {
664     final String logPrefix = "logPrefix";
665     Mocks mocks = new Mocks();
666     // set table cfs to filter all cells out
667     final TableName replicatedTable = TableName.valueOf("replicated_table");
668     final Map<TableName, List<String>> cfs =
669       Collections.singletonMap(replicatedTable, Collections.<String>emptyList());
670     when(mocks.peer.getTableCFs()).thenReturn(cfs);
671 
672     // Create a 0 length log.
673     Path emptyLog = new Path(logDir, logPrefix + ".1");
674     FSDataOutputStream fsdos = FS.create(emptyLog);
675     fsdos.close();
676     assertEquals(0, FS.getFileStatus(emptyLog).getLen());
677 
678     // Replication end point with no filter
679     final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest() {
680       @Override
681       public WALEntryFilter getWALEntryfilter() {
682         return null;
683       }
684     };
685 
686     final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint, true);
687     source.run();
688     source.enqueueLog(emptyLog);
689 
690     // Wait and verify if no entry got replicated
691     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
692       @Override public boolean evaluate() {
693         return endpoint.lastEntries == null;
694       }
695     });
696 
697     // Wait and verify get is queue is empty
698     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
699       @Override public boolean evaluate() {
700         return source.getQueues().get(logPrefix).isEmpty();
701       }
702     });
703   }
704 
705   @Test
706   public void testReplicationOnEmptyLogBetweenTheNonEmptyLogsInLogQueue() throws Exception {
707     final String logPrefix = "logPrefix";
708     Mocks mocks = new Mocks();
709     // set table cfs to filter all cells out
710     final TableName replicatedTable = TableName.valueOf("replicated_table");
711     final Map<TableName, List<String>> cfs =
712       Collections.singletonMap(replicatedTable, Collections.<String>emptyList());
713     when(mocks.peer.getTableCFs()).thenReturn(cfs);
714 
715     // Append 3 entries in a log
716     final Path log1 = new Path(logDir, logPrefix + ".11");
717     WALProvider.Writer writer1 = WALFactory.createWALWriter(FS, log1, TEST_UTIL.getConfiguration());
718     appendEntries(writer1, 3);
719 
720     // Create a 0 length log.
721     Path emptyLog = new Path(logDir, logPrefix + ".12");
722     FSDataOutputStream fsdos = FS.create(emptyLog);
723     fsdos.close();
724     assertEquals(0, FS.getFileStatus(emptyLog).getLen());
725 
726     // Append 5 entries in a log
727     final Path log3 = new Path(logDir, logPrefix + ".13");
728     WALProvider.Writer writer3 = WALFactory.createWALWriter(FS, log3, TEST_UTIL.getConfiguration());
729     appendEntries(writer3, 5);
730 
731     // Append 10 entries in a log
732     final Path log4 = new Path(logDir, logPrefix + ".14");
733     WALProvider.Writer writer4 = WALFactory.createWALWriter(FS, log4, TEST_UTIL.getConfiguration());
734     appendEntries(writer4, 10);
735 
736     // Replication end point with no filter
737     final ReplicationEndpointForTest endpoint = new ReplicationEndpointForTest() {
738       @Override
739       public WALEntryFilter getWALEntryfilter() {
740         return null;
741       }
742     };
743 
744     final ReplicationSource source = mocks.createReplicationSourceWithMocks(endpoint, true);
745     source.run();
746     source.enqueueLog(log1);
747     source.enqueueLog(emptyLog);
748     source.enqueueLog(log3);
749     source.enqueueLog(log4);
750 
751     // Wait for source to replicate
752     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
753       @Override public boolean evaluate() {
754         return endpoint.replicateCount.get() == 2;
755       }
756     });
757 
758     // Wait and verify the last replicated entries
759     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
760       @Override public boolean evaluate() {
761         return endpoint.lastEntries.size() == 15;
762       }
763     });
764 
765     // Wait and verify only one log is there in queue
766     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
767       @Override public boolean evaluate() {
768         return source.getQueues().get(logPrefix).size() == 1;
769       }
770     });
771   }
772 
773   /**
774    * Tests that recovered queues are preserved on a regionserver shutdown.
775    * See HBASE-18192
776    */
777   @Test
778   public void testServerShutdownRecoveredQueue() throws Exception {
779     try {
780       // Ensure single-threaded WAL
781       conf.set("hbase.wal.provider", "defaultProvider");
782       conf.setInt("replication.sleep.before.failover", 2000);
783       // Introduces a delay in regionserver shutdown to give the race condition a chance to kick in.
784       conf.set(HConstants.REGION_SERVER_IMPL, ShutdownDelayRegionServer.class.getName());
785       MiniHBaseCluster cluster = TEST_UTIL.startMiniCluster(2);
786       TEST_UTIL_PEER.startMiniCluster(1);
787 
788       HRegionServer serverA = cluster.getRegionServer(0);
789       final ReplicationSourceManager managerA =
790           ((Replication) serverA.getReplicationSourceService()).getReplicationManager();
791       HRegionServer serverB = cluster.getRegionServer(1);
792       final ReplicationSourceManager managerB =
793           ((Replication) serverB.getReplicationSourceService()).getReplicationManager();
794       final ReplicationAdmin replicationAdmin = new ReplicationAdmin(TEST_UTIL.getConfiguration());
795 
796       final String peerId = "TestPeer";
797       replicationAdmin.addPeer(peerId,
798           new ReplicationPeerConfig().setClusterKey(TEST_UTIL_PEER.getClusterKey()), null);
799       // Wait for replication sources to come up
800       Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
801         @Override public boolean evaluate() throws Exception {
802           return !(managerA.getSources().isEmpty() || managerB.getSources().isEmpty());
803         }
804       });
805       // Disabling peer makes sure there is at least one log to claim when the server dies
806       // The recovered queue will also stay there until the peer is disabled even if the
807       // WALs it contains have no data.
808       replicationAdmin.disablePeer(peerId);
809 
810       // Stopping serverA
811       // It's queues should be claimed by the only other alive server i.e. serverB
812       cluster.stopRegionServer(serverA.getServerName());
813       Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
814         @Override public boolean evaluate() throws Exception {
815           return managerB.getOldSources().size() == 1;
816         }
817       });
818 
819       final HRegionServer serverC = cluster.startRegionServer().getRegionServer();
820       serverC.waitForServerOnline();
821       Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
822         @Override public boolean evaluate() throws Exception {
823           return serverC.getReplicationSourceService() != null;
824         }
825       });
826       final ReplicationSourceManager managerC =
827           ((Replication) serverC.getReplicationSourceService()).getReplicationManager();
828       // Sanity check
829       assertEquals(0, managerC.getOldSources().size());
830 
831       // Stopping serverB
832       // Now serverC should have two recovered queues:
833       // 1. The serverB's normal queue
834       // 2. serverA's recovered queue on serverB
835       cluster.stopRegionServer(serverB.getServerName());
836       Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
837         @Override public boolean evaluate() throws Exception {
838           return managerC.getOldSources().size() == 2;
839         }
840       });
841       replicationAdmin.enablePeer(peerId);
842       Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
843         @Override public boolean evaluate() throws Exception {
844           return managerC.getOldSources().size() == 0;
845         }
846       });
847     } finally {
848       conf.set(HConstants.REGION_SERVER_IMPL, HRegionServer.class.getName());
849     }
850   }
851 
852   /**
853    * Regionserver implementation that adds a delay on the graceful shutdown.
854    */
855   public static class ShutdownDelayRegionServer extends HRegionServer {
856     public ShutdownDelayRegionServer(Configuration conf) throws IOException, InterruptedException {
857       super(conf);
858     }
859 
860     public ShutdownDelayRegionServer(Configuration conf, CoordinatedStateManager csm)
861         throws IOException, InterruptedException {
862       super(conf, csm);
863     }
864 
865     @Override
866     protected void stopServiceThreads() {
867       // Add a delay before service threads are shutdown.
868       // This will keep the zookeeper connection alive for the duration of the delay.
869       LOG.info("Adding a delay to the regionserver shutdown");
870       try {
871         Thread.sleep(2000);
872       } catch (InterruptedException ex) {
873         LOG.error("Interrupted while sleeping");
874       }
875       super.stopServiceThreads();
876     }
877   }
878 
879   /**
880   Test age of oldest wal metric.
881   */
882   @Test
883   public void testAgeOfOldestWal() throws Exception {
884     try {
885       ManualEnvironmentEdge manualEdge = new ManualEnvironmentEdge();
886       EnvironmentEdgeManager.injectEdge(manualEdge);
887 
888       String id = "1";
889       MetricsSource metrics = new MetricsSource(id);
890       Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
891       conf.setInt("replication.source.maxretriesmultiplier", 1);
892       Mocks mocks = new Mocks();
893       ReplicationEndpoint endpoint = mock(ReplicationEndpoint.class);
894       ReplicationSource source = mocks.createReplicationSourceWithMocks(metrics, endpoint);
895 
896       final Path log1 = new Path(logDir, "log-walgroup-a.8");
897       manualEdge.setValue(10);
898       // Diff of current time (10) and  log-walgroup-a.8 timestamp will be 2.
899       source.enqueueLog(log1);
900       MetricsReplicationSourceSource metricsSource1 = getSourceMetrics(id);
901       assertEquals(2, metricsSource1.getOldestWalAge());
902 
903       final Path log2 = new Path(logDir, "log-walgroup-b.4");
904       // Diff of current time (10) and log-walgroup-b.4 will be 6 so oldestWalAge should be 6
905       source.enqueueLog(log2);
906       assertEquals(6, metricsSource1.getOldestWalAge());
907       // Clear all metrics.
908       metrics.clear();
909     } finally {
910       EnvironmentEdgeManager.reset();
911     }
912   }
913 
914   @Test
915   public void testReplicationSourceInitializingMetric() throws Exception {
916     String id = "1";
917     MetricsSource metrics = Mockito.spy(new MetricsSource(id));
918     Mocks mocks = new Mocks();
919     ReplicationSource source = mocks.createReplicationSourceWithMocks(metrics,
920       new TestReplicationEndpoint.BadReplicationEndpoint());
921     source.startup();
922     final MetricsReplicationSourceSource metricsSource1 = getSourceMetrics(id);
923     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
924       @Override public boolean evaluate() {
925         return metricsSource1.getSourceInitializing() == 1;
926       }
927     });
928     TestReplicationEndpoint.BadReplicationEndpoint.failing = false;
929     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
930       @Override public boolean evaluate() {
931         return metricsSource1.getSourceInitializing() == 0;
932       }
933     });
934     metrics.clear();
935   }
936 
937   private MetricsReplicationSourceSource getSourceMetrics(String sourceId) {
938     MetricsReplicationSourceFactory factory = CompatibilitySingletonFactory
939       .getInstance(MetricsReplicationSourceFactory.class);
940     return factory.getSource(sourceId);
941   }
942 }