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.assertArrayEquals;
22  import static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertFalse;
24  import static org.junit.Assert.assertNotEquals;
25  import static org.junit.Assert.assertNotNull;
26  import static org.junit.Assert.assertNull;
27  import static org.junit.Assert.assertTrue;
28  import static org.junit.Assert.fail;
29  import static org.mockito.Mockito.doNothing;
30  import static org.mockito.Mockito.mock;
31  import static org.mockito.Mockito.timeout;
32  import static org.mockito.Mockito.verify;
33  import static org.mockito.Mockito.when;
34  
35  import java.io.IOException;
36  import java.util.ArrayList;
37  import java.util.Arrays;
38  import java.util.Collections;
39  import java.util.HashMap;
40  import java.util.List;
41  import java.util.Map;
42  import java.util.NavigableMap;
43  import java.util.NoSuchElementException;
44  import java.util.TreeMap;
45  import java.util.concurrent.Callable;
46  import java.util.concurrent.ExecutionException;
47  import java.util.concurrent.Executors;
48  import java.util.concurrent.Future;
49  import java.util.concurrent.PriorityBlockingQueue;
50  import java.util.concurrent.atomic.AtomicBoolean;
51  import java.util.concurrent.atomic.AtomicLong;
52  
53  import org.apache.hadoop.conf.Configuration;
54  import org.apache.hadoop.fs.FSDataOutputStream;
55  import org.apache.hadoop.fs.FileSystem;
56  import org.apache.hadoop.fs.Path;
57  import org.apache.hadoop.hbase.Cell;
58  import org.apache.hadoop.hbase.CellUtil;
59  import org.apache.hadoop.hbase.HBaseTestingUtility;
60  import org.apache.hadoop.hbase.HConstants;
61  import org.apache.hadoop.hbase.HRegionInfo;
62  import org.apache.hadoop.hbase.HTableDescriptor;
63  import org.apache.hadoop.hbase.KeyValue;
64  import org.apache.hadoop.hbase.TableName;
65  import org.apache.hadoop.hbase.Waiter;
66  import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
67  import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
68  import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
69  import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
70  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
71  import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
72  import org.apache.hadoop.hbase.replication.ReplicationPeer;
73  import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
74  import org.apache.hadoop.hbase.replication.TableCfWALEntryFilter;
75  import org.apache.hadoop.hbase.replication.WALEntryFilter;
76  import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReaderThread.WALEntryBatch;
77  import org.apache.hadoop.hbase.testclassification.LargeTests;
78  import org.apache.hadoop.hbase.testclassification.ReplicationTests;
79  import org.apache.hadoop.hbase.util.Bytes;
80  import org.apache.hadoop.hbase.wal.DefaultWALProvider;
81  import org.apache.hadoop.hbase.wal.WAL;
82  import org.apache.hadoop.hbase.wal.WAL.Entry;
83  import org.apache.hadoop.hbase.wal.WALFactory;
84  import org.apache.hadoop.hbase.wal.WALKey;
85  import org.apache.hadoop.hbase.wal.WALProvider;
86  import org.apache.hadoop.hdfs.MiniDFSCluster;
87  import org.junit.After;
88  import org.junit.AfterClass;
89  import org.junit.Before;
90  import org.junit.BeforeClass;
91  import org.junit.Rule;
92  import org.junit.Test;
93  import org.junit.experimental.categories.Category;
94  import org.junit.rules.TestName;
95  import org.junit.runner.RunWith;
96  import org.mockito.Mockito;
97  import org.mockito.invocation.InvocationOnMock;
98  import org.mockito.runners.MockitoJUnitRunner;
99  import org.mockito.stubbing.Answer;
100 
101 @RunWith(MockitoJUnitRunner.class)
102 @Category({ ReplicationTests.class, LargeTests.class })
103 public class TestWALEntryStream {
104 
105   private static HBaseTestingUtility TEST_UTIL;
106   private static Configuration conf;
107   private static FileSystem fs;
108   private static MiniDFSCluster cluster;
109   private static final TableName tableName = TableName.valueOf("tablename");
110   private static final byte[] family = Bytes.toBytes("column");
111   private static final byte[] qualifier = Bytes.toBytes("qualifier");
112   private static final HRegionInfo info =
113       new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HConstants.LAST_ROW, false);
114   private static final HTableDescriptor htd = new HTableDescriptor(tableName);
115   private static NavigableMap<byte[], Integer> scopes;
116   private final String fakeWalGroupId = "fake-wal-group-id";
117 
118   private WAL log;
119   ReplicationSourceLogQueue logQueue;
120   private PathWatcher pathWatcher;
121 
122   @Rule
123   public TestName tn = new TestName();
124   private final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
125 
126   @BeforeClass
127   public static void setUpBeforeClass() throws Exception {
128     TEST_UTIL = new HBaseTestingUtility();
129     conf = TEST_UTIL.getConfiguration();
130     TEST_UTIL.startMiniDFSCluster(3);
131 
132     cluster = TEST_UTIL.getDFSCluster();
133     fs = cluster.getFileSystem();
134     scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
135     for (byte[] fam : htd.getFamiliesKeys()) {
136       scopes.put(fam, 0);
137     }
138   }
139 
140   @AfterClass
141   public static void tearDownAfterClass() throws Exception {
142     TEST_UTIL.shutdownMiniCluster();
143   }
144 
145   @Before
146   public void setUp() throws Exception {
147     MetricsSource source = new MetricsSource("2");
148     // Source with the same id is shared and carries values from the last run
149     source.clear();
150     logQueue = new ReplicationSourceLogQueue(conf, source);
151     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
152     pathWatcher = new PathWatcher();
153     listeners.add(pathWatcher);
154     final WALFactory wals = new WALFactory(conf, listeners, tn.getMethodName());
155     log = wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
156   }
157 
158   @After
159   public void tearDown() throws Exception {
160     log.close();
161   }
162 
163   // Try out different combinations of row count and KeyValue count
164   @Test
165   public void testDifferentCounts() throws Exception {
166     int[] NB_ROWS = { 1500, 60000 };
167     int[] NB_KVS = { 1, 100 };
168     // whether compression is used
169     Boolean[] BOOL_VALS = { false, true };
170     // long lastPosition = 0;
171     for (int nbRows : NB_ROWS) {
172       for (int walEditKVs : NB_KVS) {
173         for (boolean isCompressionEnabled : BOOL_VALS) {
174           TEST_UTIL.getConfiguration().setBoolean(HConstants.ENABLE_WAL_COMPRESSION,
175             isCompressionEnabled);
176           mvcc.advanceTo(1);
177 
178           for (int i = 0; i < nbRows; i++) {
179             appendToLogPlus(walEditKVs);
180           }
181 
182           log.rollWriter();
183 
184           try (WALEntryStream entryStream =
185               new WALEntryStream(logQueue, fs, conf, new MetricsSource("1"), fakeWalGroupId)) {
186             int i = 0;
187             for (WAL.Entry e : entryStream) {
188               assertNotNull(e);
189               i++;
190             }
191             assertEquals(nbRows, i);
192 
193             // should've read all entries
194             assertFalse(entryStream.hasNext());
195           }
196           // reset everything for next loop
197           log.close();
198           setUp();
199         }
200       }
201     }
202   }
203 
204   /**
205    * Tests basic reading of log appends
206    */
207   @Test
208   public void testAppendsWithRolls() throws Exception {
209     appendToLog();
210 
211     long oldPos;
212     try (WALEntryStream entryStream =
213         new WALEntryStream(logQueue, fs, conf, new MetricsSource("1"), fakeWalGroupId)) {
214       // There's one edit in the log, read it. Reading past it needs to throw exception
215       assertTrue(entryStream.hasNext());
216       WAL.Entry entry = entryStream.next();
217       assertNotNull(entry);
218       assertFalse(entryStream.hasNext());
219       try {
220         entry = entryStream.next();
221         fail();
222       } catch (NoSuchElementException e) {
223         // expected
224       }
225       oldPos = entryStream.getPosition();
226     }
227 
228     appendToLog();
229 
230     try (WALEntryStream entryStream =
231         new WALEntryStream(logQueue, fs, conf, oldPos, new MetricsSource("1"), fakeWalGroupId)) {
232       // Read the newly added entry, make sure we made progress
233       WAL.Entry entry = entryStream.next();
234       assertNotEquals(oldPos, entryStream.getPosition());
235       assertNotNull(entry);
236       oldPos = entryStream.getPosition();
237     }
238 
239     // We rolled but we still should see the end of the first log and get that item
240     appendToLog();
241     log.rollWriter();
242     appendToLog();
243 
244     try (WALEntryStream entryStream =
245         new WALEntryStream(logQueue, fs, conf, oldPos, new MetricsSource("1"), fakeWalGroupId)) {
246       WAL.Entry entry = entryStream.next();
247       assertNotEquals(oldPos, entryStream.getPosition());
248       assertNotNull(entry);
249 
250       // next item should come from the new log
251       entry = entryStream.next();
252       assertNotEquals(oldPos, entryStream.getPosition());
253       assertNotNull(entry);
254 
255       // no more entries to read
256       assertFalse(entryStream.hasNext());
257       oldPos = entryStream.getPosition();
258     }
259   }
260 
261   /**
262    * Tests that if after a stream is opened, more entries come in and then the log is rolled, we
263    * don't mistakenly dequeue the current log thinking we're done with it
264    */
265   @Test
266   public void testLogrollWhileStreaming() throws Exception {
267     appendToLog("1");
268     appendToLog("2");// 2
269     try (WALEntryStream entryStream =
270         new WALEntryStream(logQueue, fs, conf, new MetricsSource("1"), fakeWalGroupId)) {
271       assertEquals("1", getRow(entryStream.next()));
272 
273       appendToLog("3"); // 3 - comes in after reader opened
274       log.rollWriter(); // log roll happening while we're reading
275       appendToLog("4"); // 4 - this append is in the rolled log
276 
277       assertEquals("2", getRow(entryStream.next()));
278       assertEquals(2, getQueue().size()); // we should not have dequeued yet since there's still an
279                                         // entry in first log
280       assertEquals("3", getRow(entryStream.next())); // if implemented improperly, this would be 4
281                                                      // and 3 would be skipped
282       assertEquals("4", getRow(entryStream.next())); // 4
283       assertEquals(1, getQueue().size()); // now we've dequeued and moved on to next log properly
284       assertFalse(entryStream.hasNext());
285     }
286   }
287 
288   /**
289    * Tests that if writes come in while we have a stream open, we shouldn't miss them
290    */
291   @Test
292   public void testNewEntriesWhileStreaming() throws Exception {
293     appendToLog("1");
294     try (WALEntryStream entryStream =
295         new WALEntryStream(logQueue, fs, conf, 0, new MetricsSource("1"), fakeWalGroupId)) {
296       entryStream.next(); // we've hit the end of the stream at this point
297 
298       // some new entries come in while we're streaming
299       appendToLog("2");
300       appendToLog("3");
301 
302       // don't see them
303       assertFalse(entryStream.hasNext());
304 
305       // But we do if we reset
306       entryStream.reset();
307       assertEquals("2", getRow(entryStream.next()));
308       assertEquals("3", getRow(entryStream.next()));
309       assertFalse(entryStream.hasNext());
310     }
311   }
312 
313   @Test
314   public void testResumeStreamingFromPosition() throws Exception {
315     long lastPosition = 0;
316     appendToLog("1");
317     try (WALEntryStream entryStream =
318         new WALEntryStream(logQueue, fs, conf, 0, new MetricsSource("1"), fakeWalGroupId)) {
319       entryStream.next(); // we've hit the end of the stream at this point
320       appendToLog("2");
321       appendToLog("3");
322       lastPosition = entryStream.getPosition();
323     }
324     // next stream should picks up where we left off
325     try (WALEntryStream entryStream =
326         new WALEntryStream(logQueue, fs, conf, lastPosition, new MetricsSource("1"),
327         fakeWalGroupId)) {
328       assertEquals("2", getRow(entryStream.next()));
329       assertEquals("3", getRow(entryStream.next()));
330       assertFalse(entryStream.hasNext()); // done
331       assertEquals(1, getQueue().size());
332     }
333   }
334 
335   /**
336    * Tests that if we stop before hitting the end of a stream, we can continue where we left off
337    * using the last position
338    */
339   @Test
340   public void testPosition() throws Exception {
341     long lastPosition = 0;
342     appendEntriesToLog(3);
343     // read only one element
344     try (WALEntryStream entryStream =
345         new WALEntryStream(logQueue, fs, conf, lastPosition, new MetricsSource("1"),
346         fakeWalGroupId)) {
347       entryStream.next();
348       lastPosition = entryStream.getPosition();
349     }
350     // there should still be two more entries from where we left off
351     try (WALEntryStream entryStream =
352         new WALEntryStream(logQueue, fs, conf, lastPosition, new MetricsSource("1"),
353         fakeWalGroupId)) {
354       assertNotNull(entryStream.next());
355       assertNotNull(entryStream.next());
356       assertFalse(entryStream.hasNext());
357     }
358   }
359 
360 
361   @Test
362   public void testEmptyStream() throws Exception {
363     try (WALEntryStream entryStream =
364         new WALEntryStream(logQueue, fs, conf, 0, new MetricsSource("1"), fakeWalGroupId)) {
365       assertFalse(entryStream.hasNext());
366     }
367   }
368 
369   @Test
370   public void testReplicationSourceWALReaderThread() throws Exception {
371     appendEntriesToLog(3);
372     // get ending position
373     long position;
374     try (WALEntryStream entryStream =
375         new WALEntryStream(logQueue, fs, conf, new MetricsSource("1"), fakeWalGroupId)) {
376       entryStream.next();
377       entryStream.next();
378       entryStream.next();
379       position = entryStream.getPosition();
380     }
381 
382     // start up a batcher
383     ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class);
384     ReplicationSource source = Mockito.mock(ReplicationSource.class);
385     when(source.isPeerEnabled()).thenReturn(true);
386     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
387     ReplicationSourceWALReaderThread batcher =
388             new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(),logQueue, 0,
389                     fs, conf, getDummyFilter(), new MetricsSource("1"), source, fakeWalGroupId);
390     Path walPath = getQueue().peek();
391     batcher.start();
392     WALEntryBatch entryBatch = batcher.take();
393 
394     // should've batched up our entries
395     assertNotNull(entryBatch);
396     assertEquals(3, entryBatch.getWalEntries().size());
397     assertEquals(position, entryBatch.getLastWalPosition());
398     assertEquals(walPath, entryBatch.getLastWalPath());
399     assertEquals(3, entryBatch.getNbRowKeys());
400 
401     appendToLog("foo");
402     entryBatch = batcher.take();
403     assertEquals(1, entryBatch.getNbEntries());
404     assertEquals(getRow(entryBatch.getWalEntries().get(0)), "foo");
405   }
406 
407   @Test
408   public void testReplicationSourceWALReaderWithFailingFilter() throws Exception {
409     appendEntriesToLog(3);
410     // get ending position
411     long position;
412     try (WALEntryStream entryStream =
413       new WALEntryStream(logQueue, fs, conf, new MetricsSource("1"), fakeWalGroupId)) {
414       entryStream.next();
415       entryStream.next();
416       entryStream.next();
417       position = entryStream.getPosition();
418     }
419 
420     int numFailuresInFilter = 5;
421     ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class);
422     ReplicationSource source = Mockito.mock(ReplicationSource.class);
423     when(source.isPeerEnabled()).thenReturn(true);
424     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
425     ReplicationSourceWALReaderThread batcher =
426       new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(),logQueue, 0,
427         fs, conf, getIntermittentFailingFilter(numFailuresInFilter), new MetricsSource("1"),
428         source, fakeWalGroupId);
429     Path walPath = getQueue().peek();
430     batcher.start();
431     WALEntryBatch entryBatch = batcher.take();
432     assertEquals(numFailuresInFilter, FailingWALEntryFilter.numFailures());
433 
434     // should've batched up our entries
435     assertNotNull(entryBatch);
436     assertEquals(3, entryBatch.getWalEntries().size());
437     assertEquals(position, entryBatch.getLastWalPosition());
438     assertEquals(walPath, entryBatch.getLastWalPath());
439     assertEquals(3, entryBatch.getNbRowKeys());
440   }
441 
442   @Test
443   public void testReplicationSourceWALReaderThreadRecoveredQueue() throws Exception {
444     appendEntriesToLog(3);
445     log.rollWriter();
446     appendEntriesToLog(2);
447 
448     long position;
449     ReplicationSourceLogQueue tempQueue = new ReplicationSourceLogQueue(conf,
450         getMockMetrics());
451     for (Path path : getQueue()) {
452       tempQueue.enqueueLog(path, fakeWalGroupId);
453     }
454     try (WALEntryStream entryStream = new WALEntryStream(tempQueue,
455             fs, conf, new MetricsSource("1"), fakeWalGroupId)) {
456       entryStream.next();
457       entryStream.next();
458       entryStream.next();
459       entryStream.next();
460       entryStream.next();
461       position = entryStream.getPosition();
462     }
463 
464     ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class);
465     ReplicationSource source = Mockito.mock(ReplicationSource.class);
466     when(source.isPeerEnabled()).thenReturn(true);
467     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
468     ReplicationSourceWALReaderThread reader =
469             new ReplicationSourceWALReaderThread(mockSourceManager, getRecoveredQueueInfo(),
470             logQueue, 0, fs, conf, getDummyFilter(),
471             new MetricsSource("1"), source, fakeWalGroupId);
472     Path walPath = getQueue().toArray(new Path[2])[1];
473     reader.start();
474     WALEntryBatch entryBatch = reader.take();
475 
476     assertNotNull(entryBatch);
477     assertEquals(5, entryBatch.getWalEntries().size());
478     assertEquals(position, entryBatch.getLastWalPosition());
479     assertEquals(walPath, entryBatch.getLastWalPath());
480     assertFalse(entryBatch.hasMoreEntries());
481   }
482 
483   @Test
484   public void testWALKeySerialization() throws Exception {
485     Map<String, byte[]> attributes = new HashMap<String, byte[]>();
486     attributes.put("foo", Bytes.toBytes("foo-value"));
487     attributes.put("bar", Bytes.toBytes("bar-value"));
488     WALKey key = new WALKey(info.getEncodedNameAsBytes(), tableName,
489       System.currentTimeMillis(), 0L, 0L, mvcc, attributes);
490     assertEquals(attributes, key.getExtendedAttributes());
491 
492     WALProtos.WALKey.Builder builder = key.getBuilder(null);
493     WALProtos.WALKey serializedKey = builder.build();
494 
495     WALKey deserializedKey = new WALKey();
496     deserializedKey.readFieldsFromPb(serializedKey, null);
497 
498     //equals() only checks region name, sequence id and write time
499     assertEquals(key, deserializedKey);
500     //can't use Map.equals() because byte arrays use reference equality
501     assertEquals(key.getExtendedAttributes().keySet(),
502       deserializedKey.getExtendedAttributes().keySet());
503     for (Map.Entry<String, byte[]> entry : deserializedKey.getExtendedAttributes().entrySet()) {
504       assertArrayEquals(key.getExtendedAttribute(entry.getKey()), entry.getValue());
505     }
506   }
507 
508   @Test
509   public void testReplicationSourceWALReaderThreadWithFilter() throws Exception {
510     final byte[] notReplicatedCf = Bytes.toBytes("notReplicated");
511     final Map<TableName, List<String>> tableCfs = new HashMap<>();
512     tableCfs.put(tableName, Collections.singletonList(Bytes.toString(family)));
513     ReplicationPeer peer = mock(ReplicationPeer.class);
514     when(peer.getTableCFs()).thenReturn(tableCfs);
515     WALEntryFilter filter = new ChainWALEntryFilter(new TableCfWALEntryFilter(peer));
516 
517     // add filterable entries
518     appendToLogPlus(3, notReplicatedCf);
519     appendToLogPlus(3, notReplicatedCf);
520     appendToLogPlus(3, notReplicatedCf);
521 
522     // add non filterable entries
523     appendEntriesToLog(2);
524 
525     ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class);
526     ReplicationSource source = Mockito.mock(ReplicationSource.class);
527     when(source.isPeerEnabled()).thenReturn(true);
528     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
529     final ReplicationSourceWALReaderThread reader =
530             new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(), logQueue,
531                     0, fs, conf, filter, new MetricsSource("1"), source, fakeWalGroupId);
532     reader.start();
533 
534     WALEntryBatch entryBatch = reader.take();
535 
536     assertNotNull(entryBatch);
537     assertFalse(entryBatch.isEmpty());
538     List<Entry> walEntries = entryBatch.getWalEntries();
539     assertEquals(2, walEntries.size());
540     for (Entry entry : walEntries) {
541       ArrayList<Cell> cells = entry.getEdit().getCells();
542       assertTrue(cells.size() == 1);
543       assertTrue(CellUtil.matchingFamily(cells.get(0), family));
544     }
545   }
546 
547   @Test
548   public void testReplicationSourceWALReaderThreadWithFilterWhenLogRolled() throws Exception {
549     final byte[] notReplicatedCf = Bytes.toBytes("notReplicated");
550     final Map<TableName, List<String>> tableCfs = new HashMap<>();
551     tableCfs.put(tableName, Collections.singletonList(Bytes.toString(family)));
552     ReplicationPeer peer = mock(ReplicationPeer.class);
553     when(peer.getTableCFs()).thenReturn(tableCfs);
554     WALEntryFilter filter = new ChainWALEntryFilter(new TableCfWALEntryFilter(peer));
555 
556     appendToLogPlus(3, notReplicatedCf);
557 
558     Path firstWAL = getQueue().peek();
559     final long eof = getPosition(firstWAL);
560 
561     ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class);
562     ReplicationSource source = Mockito.mock(ReplicationSource.class);
563     when(source.isPeerEnabled()).thenReturn(true);
564     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
565     final ReplicationSourceWALReaderThread reader =
566             new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(), logQueue,
567                     0, fs, conf, filter, new MetricsSource("1"), source, fakeWalGroupId);
568     reader.start();
569 
570     // reader won't put any batch, even if EOF reached.
571     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
572       @Override public boolean evaluate() {
573         return reader.getLastReadPosition() >= eof;
574       }
575     });
576     assertNull(reader.poll(0));
577 
578     log.rollWriter();
579 
580     // should get empty batch with current wal position, after wal rolled
581     WALEntryBatch entryBatch = reader.take();
582 
583     Path lastWAL= getQueue().peek();
584     long positionToBeLogged = getPosition(lastWAL);
585 
586     assertNotNull(entryBatch);
587     assertTrue(entryBatch.isEmpty());
588     assertEquals(1, getQueue().size());
589     assertNotEquals(firstWAL, entryBatch.getLastWalPath());
590     assertEquals(lastWAL, entryBatch.getLastWalPath());
591     assertEquals(positionToBeLogged, entryBatch.getLastWalPosition());
592   }
593 
594   private long getPosition(Path walPath) throws IOException {
595     ReplicationSourceLogQueue tempQueue =
596         new ReplicationSourceLogQueue(conf, getMockMetrics());
597     String walPrefix = DefaultWALProvider.getWALPrefixFromWALName(walPath.getName());
598     tempQueue.enqueueLog(walPath, walPrefix);
599     WALEntryStream entryStream =
600             new WALEntryStream(tempQueue, fs, conf, getMockMetrics(), walPrefix);
601     entryStream.hasNext();
602     return entryStream.getPosition();
603   }
604 
605   private String getRow(WAL.Entry entry) {
606     Cell cell = entry.getEdit().getCells().get(0);
607     return Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
608   }
609 
610   private void appendToLog(String key) throws IOException {
611     final long txid = log.append(htd, info,
612       new WALKey(info.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(), mvcc),
613       getWALEdit(key), true);
614     log.sync(txid);
615   }
616 
617   private void appendEntriesToLog(int count) throws IOException {
618     for (int i = 0; i < count; i++) {
619       appendToLog();
620     }
621   }
622 
623   private void appendToLog() throws IOException {
624     appendToLogPlus(1);
625   }
626 
627   private void appendToLogPlus(int count) throws IOException {
628     appendToLogPlus(count, family, qualifier);
629   }
630 
631   private void appendToLogPlus(int count, byte[] cf) throws IOException {
632     appendToLogPlus(count, cf, qualifier);
633   }
634 
635   private void appendToLogPlus(int count, byte[] cf, byte[] cq) throws IOException {
636     final long txid = log.append(htd, info,
637       new WALKey(info.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(), mvcc),
638       getWALEdits(count, cf, cq), true);
639     log.sync(txid);
640   }
641 
642   private WALEdit getWALEdits(int count, byte[] cf, byte[] cq) {
643     WALEdit edit = new WALEdit();
644     for (int i = 0; i < count; i++) {
645       edit.add(new KeyValue(Bytes.toBytes(System.currentTimeMillis()), cf, cq,
646           System.currentTimeMillis(), cq));
647     }
648     return edit;
649   }
650 
651   private WALEdit getWALEdit(String row) {
652     WALEdit edit = new WALEdit();
653     edit.add(
654       new KeyValue(Bytes.toBytes(row), family, qualifier, System.currentTimeMillis(), qualifier));
655     return edit;
656   }
657 
658   private WALEntryFilter getIntermittentFailingFilter(int numFailuresInFilter) {
659     return new FailingWALEntryFilter(numFailuresInFilter);
660   }
661 
662   public static class FailingWALEntryFilter implements WALEntryFilter {
663     private int numFailures = 0;
664     private static int countFailures = 0;
665 
666     public FailingWALEntryFilter(int numFailuresInFilter) {
667       numFailures = numFailuresInFilter;
668     }
669 
670     @Override
671     public Entry filter(Entry entry) {
672       if (countFailures == numFailures) {
673         return entry;
674       }
675       countFailures = countFailures + 1;
676       throw new WALEntryFilterRetryableException("failing filter");
677     }
678 
679     public static int numFailures(){
680       return countFailures;
681     }
682   }
683 
684   private WALEntryFilter getDummyFilter() {
685     return new WALEntryFilter() {
686 
687       @Override
688       public Entry filter(Entry entry) {
689         return entry;
690       }
691     };
692   }
693 
694   private ReplicationQueueInfo getRecoveredQueueInfo() {
695     return getQueueInfo("1-1");
696   }
697 
698   private ReplicationQueueInfo getQueueInfo() {
699     return getQueueInfo("1");
700   }
701 
702   private ReplicationQueueInfo getQueueInfo(String znode) {
703     return new ReplicationQueueInfo(znode);
704   }
705 
706   class PathWatcher extends WALActionsListener.Base {
707 
708     Path currentPath;
709 
710     @Override
711     public void preLogRoll(Path oldPath, Path newPath) {
712       logQueue.enqueueLog(newPath, fakeWalGroupId);
713       currentPath = newPath;
714     }
715   }
716 
717   @Test
718   public void testReplicationSourceWALReaderDisabled()
719     throws IOException, InterruptedException, ExecutionException {
720     for(int i=0; i<3; i++) {
721       //append and sync
722       appendToLog("key" + i);
723     }
724     // get ending position
725     long position;
726     try (WALEntryStream entryStream =
727       new WALEntryStream(logQueue, fs, conf, 0, new MetricsSource("1"), fakeWalGroupId)) {
728       entryStream.next();
729       entryStream.next();
730       entryStream.next();
731       position = entryStream.getPosition();
732     }
733 
734     // start up a reader
735     Path walPath = getQueue().peek();
736     ReplicationSource source = Mockito.mock(ReplicationSource.class);
737     when(source.getSourceMetrics()).thenReturn(new MetricsSource("1"));
738 
739     final AtomicBoolean enabled = new AtomicBoolean(false);
740     when(source.isPeerEnabled()).thenAnswer(new Answer<Boolean>() {
741       @Override
742       public Boolean answer(InvocationOnMock invocationOnMock) throws Throwable {
743         return enabled.get();
744       }
745     });
746 
747     ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class);
748     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
749     final ReplicationSourceWALReaderThread reader =
750       new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(), logQueue,
751         0, fs, conf, getDummyFilter(), new MetricsSource("1"), source, fakeWalGroupId);
752 
753     reader.start();
754     Future<WALEntryBatch> future =
755       Executors.newSingleThreadExecutor().submit(new Callable<WALEntryBatch>() {
756         @Override
757         public WALEntryBatch call() throws Exception {
758           return reader.take();
759         }
760       });
761 
762     // make sure that the isPeerEnabled has been called several times
763     verify(source, timeout(30000).atLeast(5)).isPeerEnabled();
764     // confirm that we can read nothing if the peer is disabled
765     assertFalse(future.isDone());
766     // then enable the peer, we should get the batch
767     enabled.set(true);
768     WALEntryBatch entryBatch = future.get();
769 
770     // should've batched up our entries
771     assertNotNull(entryBatch);
772     assertEquals(3, entryBatch.getWalEntries().size());
773     assertEquals(position, entryBatch.getLastWalPosition());
774     assertEquals(walPath, entryBatch.getLastWalPath());
775     assertEquals(3, entryBatch.getNbRowKeys());
776   }
777 
778   /*
779      Test removal of 0 length log from logQueue if the source is a recovered source and
780      size of logQueue is only 1.
781     */
782   @Test
783   public void testEOFExceptionForRecoveredQueue() throws Exception {
784     // Create a 0 length log.
785     Path emptyLog = new Path("emptyLog.1");
786     FSDataOutputStream fsdos = fs.create(emptyLog);
787     fsdos.close();
788     assertEquals(0, fs.getFileStatus(emptyLog).getLen());
789 
790     ReplicationSource source = Mockito.mock(ReplicationSource.class);
791 
792     ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class);
793     // Make it look like the source is from recovered source.
794     when(mockSourceManager.getOldSources())
795       .thenReturn(new ArrayList<>(Arrays.asList((ReplicationSourceInterface)source)));
796     when(source.isPeerEnabled()).thenReturn(true);
797     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
798     // Override the max retries multiplier to fail fast.
799     conf.setInt("replication.source.maxretriesmultiplier", 1);
800     conf.setBoolean("replication.source.eof.autorecovery", true);
801 
802     ReplicationSourceLogQueue localLogQueue =
803         new ReplicationSourceLogQueue(conf, getMockMetrics());
804     localLogQueue.enqueueLog(emptyLog, fakeWalGroupId);
805     // Create a reader thread.
806     ReplicationSourceWALReaderThread reader =
807         new ReplicationSourceWALReaderThread(mockSourceManager, getRecoveredQueueInfo(),
808         localLogQueue, 0, fs, conf, getDummyFilter(), getMockMetrics(), source, fakeWalGroupId);
809     reader.run();
810     assertEquals(0, localLogQueue.getQueueSize(fakeWalGroupId));
811   }
812 
813   @Test
814   public void testEOFExceptionForRecoveredQueueWithMultipleLogs() throws Exception {
815     ReplicationSourceLogQueue localLogQueue = new ReplicationSourceLogQueue(conf, getMockMetrics());
816     // Create a 0 length log.
817     Path emptyLog = new Path("log.2");
818     FSDataOutputStream fsdos = fs.create(emptyLog);
819     fsdos.close();
820     assertEquals(0, fs.getFileStatus(emptyLog).getLen());
821     localLogQueue.enqueueLog(emptyLog, fakeWalGroupId);
822 
823     final Path log1 = new Path("log.1");
824     WALProvider.Writer writer1 = WALFactory.createWALWriter(fs, log1, TEST_UTIL.getConfiguration());
825     appendEntries(writer1, 3);
826     localLogQueue.enqueueLog(log1, fakeWalGroupId);
827 
828     ReplicationSource source = Mockito.mock(ReplicationSource.class);
829     ReplicationSourceManager mockSourceManager = mock(ReplicationSourceManager.class);
830     // Make it look like the source is from recovered source.
831     when(mockSourceManager.getOldSources())
832       .thenReturn(new ArrayList<>(Arrays.asList((ReplicationSourceInterface)source)));
833     when(source.isPeerEnabled()).thenReturn(true);
834     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
835     // Override the max retries multiplier to fail fast.
836     conf.setInt("replication.source.maxretriesmultiplier", 1);
837     conf.setBoolean("replication.source.eof.autorecovery", true);
838     // Create a reader thread.
839     ReplicationSourceWALReaderThread reader =
840       new ReplicationSourceWALReaderThread(mockSourceManager, getRecoveredQueueInfo(),
841         localLogQueue, 0, fs, conf, getDummyFilter(), getMockMetrics(), source, fakeWalGroupId);
842     assertEquals("Initial log queue size is not correct",
843       2, localLogQueue.getQueueSize(fakeWalGroupId));
844     reader.run();
845 
846     // ReplicationSourceWALReaderThread#handleEofException method will
847     // remove empty log from logQueue.
848     assertEquals("Log queue should be empty", 0, localLogQueue.getQueueSize(fakeWalGroupId));
849   }
850 
851   private PriorityBlockingQueue<Path> getQueue() {
852     return logQueue.getQueue(fakeWalGroupId);
853   }
854 
855   private MetricsSource getMockMetrics() {
856     MetricsSource source = mock(MetricsSource.class);
857     doNothing().when(source).incrSizeOfLogQueue();
858     doNothing().when(source).decrSizeOfLogQueue();
859     doNothing().when(source).setOldestWalAge(Mockito.anyInt());
860     return source;
861   }
862 
863   private void appendEntries(WALProvider.Writer writer, int numEntries) throws IOException {
864     for (int i = 0; i < numEntries; i++) {
865       byte[] b = Bytes.toBytes(Integer.toString(i));
866       KeyValue kv = new KeyValue(b,b,b);
867       WALEdit edit = new WALEdit();
868       edit.add(kv);
869       WALKey key = new WALKey(b, TableName.valueOf(b), 0, 0,
870         HConstants.DEFAULT_CLUSTER_ID);
871       NavigableMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
872       scopes.put(b, HConstants.REPLICATION_SCOPE_GLOBAL);
873       key.setScopes(scopes);
874       writer.append(new WAL.Entry(key, edit));
875       writer.sync(false);
876     }
877     writer.close();
878   }
879 
880   /**
881    * Tests size of log queue is incremented and decremented properly.
882    */
883   @Test
884   public void testSizeOfLogQueue() throws Exception {
885     // There should be always 1 log which is current wal.
886     assertEquals(1, logQueue.getMetrics().getSizeOfLogQueue());
887     appendToLog();
888     log.rollWriter();
889     // After rolling there will be 2 wals in the queue
890     assertEquals(2, logQueue.getMetrics().getSizeOfLogQueue());
891 
892     try (WALEntryStream entryStream =
893            new WALEntryStream(logQueue, fs, conf, logQueue.getMetrics(), fakeWalGroupId)) {
894       // There's one edit in the log, read it.
895       assertTrue(entryStream.hasNext());
896       WAL.Entry entry = entryStream.next();
897       assertNotNull(entry);
898       assertFalse(entryStream.hasNext());
899     }
900     // After removing one wal, size of log queue will be 1 again.
901     assertEquals(1, logQueue.getMetrics().getSizeOfLogQueue());
902   }
903 
904   /**
905    * Tests that wals are closed cleanly and we read the trailer when we remove wal
906    * from WALEntryStream.
907    */
908   @Test
909   public void testCleanClosedWALs() throws Exception {
910     try (WALEntryStream entryStream = new WALEntryStream(
911       logQueue, fs, conf, logQueue.getMetrics(), fakeWalGroupId)) {
912       assertEquals(0, logQueue.getMetrics().getUncleanlyClosedWALs());
913       appendToLog();
914       assertNotNull(entryStream.next());
915       log.rollWriter();
916       appendToLog();
917       assertNotNull(entryStream.next());
918       assertEquals(0, logQueue.getMetrics().getUncleanlyClosedWALs());
919     }
920   }
921 
922   /**
923    * Tests that we handle EOFException properly if the wal has moved to oldWALs directory.
924    * @throws Exception exception
925    */
926   @Test
927   public void testEOFExceptionInOldWALsDirectory() throws Exception {
928     assertEquals(1, logQueue.getQueueSize(fakeWalGroupId));
929     FSHLog fsLog = (FSHLog)log;
930     Path emptyLogFile = fsLog.getCurrentFileName();
931     log.rollWriter(true);
932     // There will 2 logs in the queue.
933     assertEquals(2, logQueue.getQueueSize(fakeWalGroupId));
934 
935     Configuration localConf = new Configuration(conf);
936     localConf.setInt("replication.source.maxretriesmultiplier", 1);
937     localConf.setBoolean("replication.source.eof.autorecovery", true);
938 
939     try (WALEntryStream entryStream =
940       new WALEntryStream(logQueue, fs, localConf, logQueue.getMetrics(), fakeWalGroupId)) {
941       // Get the archived dir path for the first wal.
942       Path archivePath = entryStream.getArchivedLog(emptyLogFile);
943       // Make sure that the wal path is not the same as archived Dir path.
944       assertNotEquals(emptyLogFile.toString(), archivePath.toString());
945       assertTrue(fs.exists(archivePath));
946       fs.truncate(archivePath, 0);
947       // make sure the size of the wal file is 0.
948       assertEquals(0, fs.getFileStatus(archivePath).getLen());
949     }
950 
951     ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class);
952     ReplicationSource source = Mockito.mock(ReplicationSource.class);
953     when(source.isPeerEnabled()).thenReturn(true);
954     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
955 
956     // Start the reader thread.
957     ReplicationSourceWALReaderThread readerThread =
958       new ReplicationSourceWALReaderThread(mockSourceManager, getQueueInfo(), logQueue, 0,
959         fs, localConf, getDummyFilter(), logQueue.getMetrics(), source, fakeWalGroupId);
960     readerThread.start();
961     // Wait for the replication queue size to be 1. This means that we have handled
962     // 0 length wal from oldWALs directory.
963     Waiter.waitFor(conf, 10000, new Waiter.Predicate<Exception>() {
964       @Override public boolean evaluate() {
965         return logQueue.getQueueSize(fakeWalGroupId) == 1;
966       }
967     });
968   }
969 }