001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.replication.regionserver; 019 020import static org.junit.Assert.assertEquals; 021import static org.junit.Assert.assertFalse; 022import static org.junit.Assert.assertNotEquals; 023import static org.junit.Assert.assertNotNull; 024import static org.junit.Assert.assertNull; 025import static org.junit.Assert.assertSame; 026import static org.junit.Assert.assertTrue; 027import static org.mockito.Mockito.when; 028 029import java.io.IOException; 030import java.util.ArrayList; 031import java.util.HashMap; 032import java.util.Map; 033import java.util.NavigableMap; 034import java.util.OptionalLong; 035import java.util.TreeMap; 036import java.util.UUID; 037import java.util.concurrent.ExecutionException; 038import java.util.concurrent.ForkJoinPool; 039import java.util.concurrent.Future; 040import java.util.concurrent.PriorityBlockingQueue; 041import java.util.concurrent.atomic.AtomicBoolean; 042import java.util.concurrent.atomic.AtomicInteger; 043import java.util.concurrent.atomic.AtomicLong; 044import org.apache.hadoop.conf.Configuration; 045import org.apache.hadoop.fs.FileSystem; 046import org.apache.hadoop.fs.Path; 047import org.apache.hadoop.hbase.Cell; 048import org.apache.hadoop.hbase.HBaseClassTestRule; 049import org.apache.hadoop.hbase.HBaseTestingUtility; 050import org.apache.hadoop.hbase.HConstants; 051import org.apache.hadoop.hbase.KeyValue; 052import org.apache.hadoop.hbase.Server; 053import org.apache.hadoop.hbase.TableName; 054import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; 055import org.apache.hadoop.hbase.client.RegionInfo; 056import org.apache.hadoop.hbase.client.RegionInfoBuilder; 057import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; 058import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; 059import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; 060import org.apache.hadoop.hbase.replication.WALEntryFilter; 061import org.apache.hadoop.hbase.testclassification.LargeTests; 062import org.apache.hadoop.hbase.testclassification.ReplicationTests; 063import org.apache.hadoop.hbase.util.Bytes; 064import org.apache.hadoop.hbase.wal.WAL; 065import org.apache.hadoop.hbase.wal.WAL.Entry; 066import org.apache.hadoop.hbase.wal.WALEdit; 067import org.apache.hadoop.hbase.wal.WALFactory; 068import org.apache.hadoop.hbase.wal.WALKeyImpl; 069import org.apache.hadoop.hdfs.MiniDFSCluster; 070import org.junit.After; 071import org.junit.AfterClass; 072import org.junit.Assert; 073import org.junit.Before; 074import org.junit.BeforeClass; 075import org.junit.ClassRule; 076import org.junit.Rule; 077import org.junit.Test; 078import org.junit.experimental.categories.Category; 079import org.junit.rules.TestName; 080import org.mockito.Mockito; 081import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; 082 083 084@Category({ ReplicationTests.class, LargeTests.class }) 085public class TestWALEntryStream { 086 087 @ClassRule 088 public static final HBaseClassTestRule CLASS_RULE = 089 HBaseClassTestRule.forClass(TestWALEntryStream.class); 090 091 private static HBaseTestingUtility TEST_UTIL; 092 private static Configuration CONF; 093 private static FileSystem fs; 094 private static MiniDFSCluster cluster; 095 private static final TableName tableName = TableName.valueOf("tablename"); 096 private static final byte[] family = Bytes.toBytes("column"); 097 private static final byte[] qualifier = Bytes.toBytes("qualifier"); 098 private static final RegionInfo info = RegionInfoBuilder.newBuilder(tableName) 099 .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.LAST_ROW).build(); 100 private static final NavigableMap<byte[], Integer> scopes = getScopes(); 101 102 private static NavigableMap<byte[], Integer> getScopes() { 103 NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); 104 scopes.put(family, 1); 105 return scopes; 106 } 107 108 private WAL log; 109 PriorityBlockingQueue<Path> walQueue; 110 private PathWatcher pathWatcher; 111 112 @Rule 113 public TestName tn = new TestName(); 114 private final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); 115 116 @BeforeClass 117 public static void setUpBeforeClass() throws Exception { 118 TEST_UTIL = new HBaseTestingUtility(); 119 CONF = TEST_UTIL.getConfiguration(); 120 TEST_UTIL.startMiniDFSCluster(3); 121 122 cluster = TEST_UTIL.getDFSCluster(); 123 fs = cluster.getFileSystem(); 124 } 125 126 @AfterClass 127 public static void tearDownAfterClass() throws Exception { 128 TEST_UTIL.shutdownMiniCluster(); 129 } 130 131 @Before 132 public void setUp() throws Exception { 133 walQueue = new PriorityBlockingQueue<>(); 134 pathWatcher = new PathWatcher(); 135 final WALFactory wals = new WALFactory(CONF, tn.getMethodName()); 136 wals.getWALProvider().addWALActionsListener(pathWatcher); 137 log = wals.getWAL(info); 138 } 139 140 @After 141 public void tearDown() throws Exception { 142 log.close(); 143 } 144 145 // Try out different combinations of row count and KeyValue count 146 @Test 147 public void testDifferentCounts() throws Exception { 148 int[] NB_ROWS = { 1500, 60000 }; 149 int[] NB_KVS = { 1, 100 }; 150 // whether compression is used 151 Boolean[] BOOL_VALS = { false, true }; 152 // long lastPosition = 0; 153 for (int nbRows : NB_ROWS) { 154 for (int walEditKVs : NB_KVS) { 155 for (boolean isCompressionEnabled : BOOL_VALS) { 156 TEST_UTIL.getConfiguration().setBoolean(HConstants.ENABLE_WAL_COMPRESSION, 157 isCompressionEnabled); 158 mvcc.advanceTo(1); 159 160 for (int i = 0; i < nbRows; i++) { 161 appendToLogAndSync(walEditKVs); 162 } 163 164 log.rollWriter(); 165 166 try (WALEntryStream entryStream = 167 new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) { 168 int i = 0; 169 while (entryStream.hasNext()) { 170 assertNotNull(entryStream.next()); 171 i++; 172 } 173 assertEquals(nbRows, i); 174 175 // should've read all entries 176 assertFalse(entryStream.hasNext()); 177 } 178 // reset everything for next loop 179 log.close(); 180 setUp(); 181 } 182 } 183 } 184 } 185 186 /** 187 * Tests basic reading of log appends 188 */ 189 @Test 190 public void testAppendsWithRolls() throws Exception { 191 appendToLogAndSync(); 192 long oldPos; 193 try (WALEntryStream entryStream = 194 new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) { 195 // There's one edit in the log, read it. Reading past it needs to throw exception 196 assertTrue(entryStream.hasNext()); 197 WAL.Entry entry = entryStream.peek(); 198 assertSame(entry, entryStream.next()); 199 assertNotNull(entry); 200 assertFalse(entryStream.hasNext()); 201 assertNull(entryStream.peek()); 202 assertNull(entryStream.next()); 203 oldPos = entryStream.getPosition(); 204 } 205 206 appendToLogAndSync(); 207 208 try (WALEntryStream entryStream = new WALEntryStream(walQueue, CONF, oldPos, 209 log, null, new MetricsSource("1"))) { 210 // Read the newly added entry, make sure we made progress 211 WAL.Entry entry = entryStream.next(); 212 assertNotEquals(oldPos, entryStream.getPosition()); 213 assertNotNull(entry); 214 oldPos = entryStream.getPosition(); 215 } 216 217 // We rolled but we still should see the end of the first log and get that item 218 appendToLogAndSync(); 219 log.rollWriter(); 220 appendToLogAndSync(); 221 222 try (WALEntryStream entryStream = new WALEntryStream(walQueue, CONF, oldPos, 223 log, null, new MetricsSource("1"))) { 224 WAL.Entry entry = entryStream.next(); 225 assertNotEquals(oldPos, entryStream.getPosition()); 226 assertNotNull(entry); 227 228 // next item should come from the new log 229 entry = entryStream.next(); 230 assertNotEquals(oldPos, entryStream.getPosition()); 231 assertNotNull(entry); 232 233 // no more entries to read 234 assertFalse(entryStream.hasNext()); 235 oldPos = entryStream.getPosition(); 236 } 237 } 238 239 /** 240 * Tests that if after a stream is opened, more entries come in and then the log is rolled, we 241 * don't mistakenly dequeue the current log thinking we're done with it 242 */ 243 @Test 244 public void testLogrollWhileStreaming() throws Exception { 245 appendToLog("1"); 246 appendToLog("2");// 2 247 try (WALEntryStream entryStream = 248 new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) { 249 assertEquals("1", getRow(entryStream.next())); 250 251 appendToLog("3"); // 3 - comes in after reader opened 252 log.rollWriter(); // log roll happening while we're reading 253 appendToLog("4"); // 4 - this append is in the rolled log 254 255 assertEquals("2", getRow(entryStream.next())); 256 assertEquals(2, walQueue.size()); // we should not have dequeued yet since there's still an 257 // entry in first log 258 assertEquals("3", getRow(entryStream.next())); // if implemented improperly, this would be 4 259 // and 3 would be skipped 260 assertEquals("4", getRow(entryStream.next())); // 4 261 assertEquals(1, walQueue.size()); // now we've dequeued and moved on to next log properly 262 assertFalse(entryStream.hasNext()); 263 } 264 } 265 266 /** 267 * Tests that if writes come in while we have a stream open, we shouldn't miss them 268 */ 269 @Test 270 public void testNewEntriesWhileStreaming() throws Exception { 271 appendToLog("1"); 272 try (WALEntryStream entryStream = 273 new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) { 274 entryStream.next(); // we've hit the end of the stream at this point 275 276 // some new entries come in while we're streaming 277 appendToLog("2"); 278 appendToLog("3"); 279 280 // don't see them 281 assertFalse(entryStream.hasNext()); 282 283 // But we do if we reset 284 entryStream.reset(); 285 assertEquals("2", getRow(entryStream.next())); 286 assertEquals("3", getRow(entryStream.next())); 287 assertFalse(entryStream.hasNext()); 288 } 289 } 290 291 @Test 292 public void testResumeStreamingFromPosition() throws Exception { 293 long lastPosition = 0; 294 appendToLog("1"); 295 try (WALEntryStream entryStream = 296 new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) { 297 entryStream.next(); // we've hit the end of the stream at this point 298 appendToLog("2"); 299 appendToLog("3"); 300 lastPosition = entryStream.getPosition(); 301 } 302 // next stream should picks up where we left off 303 try (WALEntryStream entryStream = 304 new WALEntryStream(walQueue, CONF, lastPosition, log, null, new MetricsSource("1"))) { 305 assertEquals("2", getRow(entryStream.next())); 306 assertEquals("3", getRow(entryStream.next())); 307 assertFalse(entryStream.hasNext()); // done 308 assertEquals(1, walQueue.size()); 309 } 310 } 311 312 /** 313 * Tests that if we stop before hitting the end of a stream, we can continue where we left off 314 * using the last position 315 */ 316 @Test 317 public void testPosition() throws Exception { 318 long lastPosition = 0; 319 appendEntriesToLogAndSync(3); 320 // read only one element 321 try (WALEntryStream entryStream = new WALEntryStream(walQueue, CONF, lastPosition, 322 log, null, new MetricsSource("1"))) { 323 entryStream.next(); 324 lastPosition = entryStream.getPosition(); 325 } 326 // there should still be two more entries from where we left off 327 try (WALEntryStream entryStream = 328 new WALEntryStream(walQueue, CONF, lastPosition, log, null, new MetricsSource("1"))) { 329 assertNotNull(entryStream.next()); 330 assertNotNull(entryStream.next()); 331 assertFalse(entryStream.hasNext()); 332 } 333 } 334 335 336 @Test 337 public void testEmptyStream() throws Exception { 338 try (WALEntryStream entryStream = 339 new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) { 340 assertFalse(entryStream.hasNext()); 341 } 342 } 343 344 @Test 345 public void testWALKeySerialization() throws Exception { 346 Map<String, byte[]> attributes = new HashMap<String, byte[]>(); 347 attributes.put("foo", Bytes.toBytes("foo-value")); 348 attributes.put("bar", Bytes.toBytes("bar-value")); 349 WALKeyImpl key = new WALKeyImpl(info.getEncodedNameAsBytes(), tableName, 350 System.currentTimeMillis(), new ArrayList<UUID>(), 0L, 0L, 351 mvcc, scopes, attributes); 352 Assert.assertEquals(attributes, key.getExtendedAttributes()); 353 354 WALProtos.WALKey.Builder builder = key.getBuilder(WALCellCodec.getNoneCompressor()); 355 WALProtos.WALKey serializedKey = builder.build(); 356 357 WALKeyImpl deserializedKey = new WALKeyImpl(); 358 deserializedKey.readFieldsFromPb(serializedKey, WALCellCodec.getNoneUncompressor()); 359 360 //equals() only checks region name, sequence id and write time 361 Assert.assertEquals(key, deserializedKey); 362 //can't use Map.equals() because byte arrays use reference equality 363 Assert.assertEquals(key.getExtendedAttributes().keySet(), 364 deserializedKey.getExtendedAttributes().keySet()); 365 for (Map.Entry<String, byte[]> entry : deserializedKey.getExtendedAttributes().entrySet()){ 366 Assert.assertArrayEquals(key.getExtendedAttribute(entry.getKey()), entry.getValue()); 367 } 368 Assert.assertEquals(key.getReplicationScopes(), deserializedKey.getReplicationScopes()); 369 } 370 371 private ReplicationSource mockReplicationSource(boolean recovered, Configuration conf) { 372 ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class); 373 when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0)); 374 Server mockServer = Mockito.mock(Server.class); 375 ReplicationSource source = Mockito.mock(ReplicationSource.class); 376 when(source.getSourceManager()).thenReturn(mockSourceManager); 377 when(source.getSourceMetrics()).thenReturn(new MetricsSource("1")); 378 when(source.getWALFileLengthProvider()).thenReturn(log); 379 when(source.getServer()).thenReturn(mockServer); 380 when(source.isRecovered()).thenReturn(recovered); 381 return source; 382 } 383 384 private ReplicationSourceWALReader createReader(boolean recovered, Configuration conf) { 385 ReplicationSource source = mockReplicationSource(recovered, conf); 386 when(source.isPeerEnabled()).thenReturn(true); 387 ReplicationSourceWALReader reader = 388 new ReplicationSourceWALReader(fs, conf, walQueue, 0, getDummyFilter(), source); 389 reader.start(); 390 return reader; 391 } 392 393 @Test 394 public void testReplicationSourceWALReader() throws Exception { 395 appendEntriesToLogAndSync(3); 396 // get ending position 397 long position; 398 try (WALEntryStream entryStream = 399 new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) { 400 entryStream.next(); 401 entryStream.next(); 402 entryStream.next(); 403 position = entryStream.getPosition(); 404 } 405 406 // start up a reader 407 Path walPath = walQueue.peek(); 408 ReplicationSourceWALReader reader = createReader(false, CONF); 409 WALEntryBatch entryBatch = reader.take(); 410 411 // should've batched up our entries 412 assertNotNull(entryBatch); 413 assertEquals(3, entryBatch.getWalEntries().size()); 414 assertEquals(position, entryBatch.getLastWalPosition()); 415 assertEquals(walPath, entryBatch.getLastWalPath()); 416 assertEquals(3, entryBatch.getNbRowKeys()); 417 418 appendToLog("foo"); 419 entryBatch = reader.take(); 420 assertEquals(1, entryBatch.getNbEntries()); 421 assertEquals("foo", getRow(entryBatch.getWalEntries().get(0))); 422 } 423 424 @Test 425 public void testReplicationSourceWALReaderRecovered() throws Exception { 426 appendEntriesToLogAndSync(10); 427 Path walPath = walQueue.peek(); 428 log.rollWriter(); 429 appendEntriesToLogAndSync(5); 430 log.shutdown(); 431 432 Configuration conf = new Configuration(CONF); 433 conf.setInt("replication.source.nb.capacity", 10); 434 435 ReplicationSourceWALReader reader = createReader(true, conf); 436 437 WALEntryBatch batch = reader.take(); 438 assertEquals(walPath, batch.getLastWalPath()); 439 assertEquals(10, batch.getNbEntries()); 440 assertFalse(batch.isEndOfFile()); 441 442 batch = reader.take(); 443 assertEquals(walPath, batch.getLastWalPath()); 444 assertEquals(0, batch.getNbEntries()); 445 assertTrue(batch.isEndOfFile()); 446 447 walPath = walQueue.peek(); 448 batch = reader.take(); 449 assertEquals(walPath, batch.getLastWalPath()); 450 assertEquals(5, batch.getNbEntries()); 451 // Actually this should be true but we haven't handled this yet since for a normal queue the 452 // last one is always open... Not a big deal for now. 453 assertFalse(batch.isEndOfFile()); 454 455 assertSame(WALEntryBatch.NO_MORE_DATA, reader.take()); 456 } 457 458 // Testcase for HBASE-20206 459 @Test 460 public void testReplicationSourceWALReaderWrongPosition() throws Exception { 461 appendEntriesToLogAndSync(1); 462 Path walPath = walQueue.peek(); 463 log.rollWriter(); 464 appendEntriesToLogAndSync(20); 465 TEST_UTIL.waitFor(5000, new ExplainingPredicate<Exception>() { 466 467 @Override 468 public boolean evaluate() throws Exception { 469 return fs.getFileStatus(walPath).getLen() > 0; 470 } 471 472 @Override 473 public String explainFailure() throws Exception { 474 return walPath + " has not been closed yet"; 475 } 476 477 }); 478 long walLength = fs.getFileStatus(walPath).getLen(); 479 480 ReplicationSourceWALReader reader = createReader(false, CONF); 481 482 WALEntryBatch entryBatch = reader.take(); 483 assertEquals(walPath, entryBatch.getLastWalPath()); 484 assertTrue("Position " + entryBatch.getLastWalPosition() + " is out of range, file length is " + 485 walLength, entryBatch.getLastWalPosition() <= walLength); 486 assertEquals(1, entryBatch.getNbEntries()); 487 assertTrue(entryBatch.isEndOfFile()); 488 489 Path walPath2 = walQueue.peek(); 490 entryBatch = reader.take(); 491 assertEquals(walPath2, entryBatch.getLastWalPath()); 492 assertEquals(20, entryBatch.getNbEntries()); 493 assertFalse(entryBatch.isEndOfFile()); 494 495 log.rollWriter(); 496 appendEntriesToLogAndSync(10); 497 entryBatch = reader.take(); 498 assertEquals(walPath2, entryBatch.getLastWalPath()); 499 assertEquals(0, entryBatch.getNbEntries()); 500 assertTrue(entryBatch.isEndOfFile()); 501 502 Path walPath3 = walQueue.peek(); 503 entryBatch = reader.take(); 504 assertEquals(walPath3, entryBatch.getLastWalPath()); 505 assertEquals(10, entryBatch.getNbEntries()); 506 assertFalse(entryBatch.isEndOfFile()); 507 } 508 509 @Test 510 public void testReplicationSourceWALReaderDisabled() 511 throws IOException, InterruptedException, ExecutionException { 512 appendEntriesToLogAndSync(3); 513 // get ending position 514 long position; 515 try (WALEntryStream entryStream = 516 new WALEntryStream(walQueue, CONF, 0, log, null, new MetricsSource("1"))) { 517 entryStream.next(); 518 entryStream.next(); 519 entryStream.next(); 520 position = entryStream.getPosition(); 521 } 522 523 // start up a reader 524 Path walPath = walQueue.peek(); 525 ReplicationSource source = mockReplicationSource(false, CONF); 526 AtomicInteger invokeCount = new AtomicInteger(0); 527 AtomicBoolean enabled = new AtomicBoolean(false); 528 when(source.isPeerEnabled()).then(i -> { 529 invokeCount.incrementAndGet(); 530 return enabled.get(); 531 }); 532 533 ReplicationSourceWALReader reader = 534 new ReplicationSourceWALReader(fs, CONF, walQueue, 0, getDummyFilter(), source); 535 reader.start(); 536 Future<WALEntryBatch> future = ForkJoinPool.commonPool().submit(() -> { 537 return reader.take(); 538 }); 539 // make sure that the isPeerEnabled has been called several times 540 TEST_UTIL.waitFor(30000, () -> invokeCount.get() >= 5); 541 // confirm that we can read nothing if the peer is disabled 542 assertFalse(future.isDone()); 543 // then enable the peer, we should get the batch 544 enabled.set(true); 545 WALEntryBatch entryBatch = future.get(); 546 547 // should've batched up our entries 548 assertNotNull(entryBatch); 549 assertEquals(3, entryBatch.getWalEntries().size()); 550 assertEquals(position, entryBatch.getLastWalPosition()); 551 assertEquals(walPath, entryBatch.getLastWalPath()); 552 assertEquals(3, entryBatch.getNbRowKeys()); 553 } 554 555 private String getRow(WAL.Entry entry) { 556 Cell cell = entry.getEdit().getCells().get(0); 557 return Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); 558 } 559 560 private void appendToLog(String key) throws IOException { 561 final long txid = log.appendData(info, 562 new WALKeyImpl(info.getEncodedNameAsBytes(), tableName, System.currentTimeMillis(), 563 mvcc, scopes), getWALEdit(key)); 564 log.sync(txid); 565 } 566 567 private void appendEntriesToLogAndSync(int count) throws IOException { 568 long txid = -1L; 569 for (int i = 0; i < count; i++) { 570 txid = appendToLog(1); 571 } 572 log.sync(txid); 573 } 574 575 private void appendToLogAndSync() throws IOException { 576 appendToLogAndSync(1); 577 } 578 579 private void appendToLogAndSync(int count) throws IOException { 580 long txid = appendToLog(count); 581 log.sync(txid); 582 } 583 584 private long appendToLog(int count) throws IOException { 585 return log.appendData(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName, 586 System.currentTimeMillis(), mvcc, scopes), getWALEdits(count)); 587 } 588 589 private WALEdit getWALEdits(int count) { 590 WALEdit edit = new WALEdit(); 591 for (int i = 0; i < count; i++) { 592 edit.add(new KeyValue(Bytes.toBytes(System.currentTimeMillis()), family, qualifier, 593 System.currentTimeMillis(), qualifier)); 594 } 595 return edit; 596 } 597 598 private WALEdit getWALEdit(String row) { 599 WALEdit edit = new WALEdit(); 600 edit.add( 601 new KeyValue(Bytes.toBytes(row), family, qualifier, System.currentTimeMillis(), qualifier)); 602 return edit; 603 } 604 605 private WALEntryFilter getDummyFilter() { 606 return new WALEntryFilter() { 607 608 @Override 609 public Entry filter(Entry entry) { 610 return entry; 611 } 612 }; 613 } 614 615 class PathWatcher implements WALActionsListener { 616 617 Path currentPath; 618 619 @Override 620 public void preLogRoll(Path oldPath, Path newPath) throws IOException { 621 walQueue.add(newPath); 622 currentPath = newPath; 623 } 624 } 625 626 @Test 627 public void testReadBeyondCommittedLength() throws IOException, InterruptedException { 628 appendToLog("1"); 629 appendToLog("2"); 630 long size = log.getLogFileSizeIfBeingWritten(walQueue.peek()).getAsLong(); 631 AtomicLong fileLength = new AtomicLong(size - 1); 632 try (WALEntryStream entryStream = new WALEntryStream(walQueue, CONF, 0, 633 p -> OptionalLong.of(fileLength.get()), null, new MetricsSource("1"))) { 634 assertTrue(entryStream.hasNext()); 635 assertNotNull(entryStream.next()); 636 // can not get log 2 637 assertFalse(entryStream.hasNext()); 638 Thread.sleep(1000); 639 entryStream.reset(); 640 // still can not get log 2 641 assertFalse(entryStream.hasNext()); 642 643 // can get log 2 now 644 fileLength.set(size); 645 entryStream.reset(); 646 assertTrue(entryStream.hasNext()); 647 assertNotNull(entryStream.next()); 648 649 assertFalse(entryStream.hasNext()); 650 } 651 } 652}