/** * Test that when no sources are registered, no side effects are observed */ @Test public void testNoRegisteredSource() throws IOException, InterruptedException { WatermarkStorage mockWatermarkStorage = mock(WatermarkStorage.class); MultiWriterWatermarkManager watermarkManager = new MultiWriterWatermarkManager(mockWatermarkStorage, 1000, Optional.<Logger>absent()); try { watermarkManager.start(); } catch (Exception e) { Assert.fail("Should not throw exception", e); } Thread.sleep(2000); watermarkManager.close(); verify(mockWatermarkStorage, times(0)).commitWatermarks(any(Iterable.class)); MultiWriterWatermarkManager.CommitStatus watermarkMgrStatus = watermarkManager.getCommitStatus(); Assert.assertTrue(watermarkMgrStatus.getLastCommittedWatermarks().isEmpty(), "Last committed watermarks should be empty"); Assert.assertEquals(watermarkMgrStatus.getLastWatermarkCommitSuccessTimestampMillis(), 0 , "Last committed watermark timestamp should be 0"); }
/** * Test that in the presence of flaky Watermark writers, we continue to log retrieval status correctly */ @Test public void testRetrievalStatus() throws InterruptedException, IOException { WatermarkStorage mockWatermarkStorage = mock(WatermarkStorage.class); MultiWriterWatermarkManager watermarkManager = new MultiWriterWatermarkManager(mockWatermarkStorage, 1000, Optional.<Logger>absent()); watermarkManager.registerWriter(getFlakyWatermarkWriter(2)); try { watermarkManager.start(); } catch (Exception e) { Assert.fail("Should not throw exception", e); } Thread.sleep(2000); watermarkManager.close(); MultiWriterWatermarkManager.RetrievalStatus retrievalStatus = watermarkManager.getRetrievalStatus(); Assert.assertTrue(retrievalStatus.getLastWatermarkRetrievalAttemptTimestampMillis() > 0); Assert.assertTrue(retrievalStatus.getLastWatermarkRetrievalSuccessTimestampMillis() > 0); Assert.assertTrue(retrievalStatus.getLastWatermarkRetrievalFailureTimestampMillis() > 0); System.out.println(retrievalStatus); }
/** * Test that when we have commits failing to watermark storage, the manager continues to try * at every interval and keeps track of the exception it is seeing. */ @Test public void testFailingWatermarkStorage() throws IOException, InterruptedException { WatermarkStorage reallyBadWatermarkStorage = mock(WatermarkStorage.class); IOException exceptionToThrow = new IOException("Failed to write coz the programmer told me to"); doThrow(exceptionToThrow).when(reallyBadWatermarkStorage).commitWatermarks(any(Iterable.class)); long commitInterval = 1000; MultiWriterWatermarkManager watermarkManager = new MultiWriterWatermarkManager(reallyBadWatermarkStorage, commitInterval, Optional.<Logger>absent()); WatermarkAwareWriter mockWriter = mock(WatermarkAwareWriter.class); CheckpointableWatermark watermark = new DefaultCheckpointableWatermark("default", new LongWatermark(0)); when(mockWriter.getCommittableWatermark()).thenReturn(Collections.singletonMap("default", watermark)); watermarkManager.registerWriter(mockWriter); try { watermarkManager.start(); } catch (Exception e) { Assert.fail("Should not throw exception", e); } Thread.sleep(commitInterval * 2 + (commitInterval/2)); // sleep for 2.5 iterations watermarkManager.close(); int expectedCalls = 3; // 2 calls from iterations, 1 additional attempt due to close verify(reallyBadWatermarkStorage, atLeast(expectedCalls)).commitWatermarks(any(Iterable.class)); Assert.assertEquals(watermarkManager.getCommitStatus().getLastCommitException(), exceptionToThrow, "Testing tracking of failed exceptions"); }
watermarkManager.close();