@Test public void testFileSuffixGiven() throws IOException, InterruptedException { final int ROLL_INTERVAL = 1000; // seconds. Make sure it doesn't change in course of test final String suffix = ".avro"; MockHDFSWriter hdfsWriter = new MockHDFSWriter(); BucketWriter bucketWriter = new BucketWriter( ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", suffix, null, SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); // Need to override system time use for test so we know what to expect final long testTime = System.currentTimeMillis(); Clock testClock = new Clock() { public long currentTimeMillis() { return testTime; } }; bucketWriter.setClock(testClock); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); Assert.assertTrue( "Incorrect suffix", hdfsWriter.getOpenedFilePath().endsWith(Long.toString(testTime + 1) + suffix + ".tmp")); }
@Test public void testInUseSuffix() throws IOException, InterruptedException { final int ROLL_INTERVAL = 1000; // seconds. Make sure it doesn't change in course of test final String SUFFIX = "WELCOME_TO_THE_HELLMOUNTH"; MockHDFSWriter hdfsWriter = new MockHDFSWriter(); HDFSTextSerializer serializer = new HDFSTextSerializer(); BucketWriter bucketWriter = new BucketWriter( ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", SUFFIX, null, null, SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); bucketWriter.append(e); Assert.assertTrue("Incorrect in use suffix", hdfsWriter.getOpenedFilePath().contains(SUFFIX)); }
@Test public void testEventCountingRoller() throws IOException, InterruptedException { int maxEvents = 100; MockHDFSWriter hdfsWriter = new MockHDFSWriter(); BucketWriter bucketWriter = new BucketWriter( 0, 0, maxEvents, 0, ctx, "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); for (int i = 0; i < 1000; i++) { bucketWriter.append(e); } logger.info("Number of events written: {}", hdfsWriter.getEventsWritten()); logger.info("Number of bytes written: {}", hdfsWriter.getBytesWritten()); logger.info("Number of files opened: {}", hdfsWriter.getFilesOpened()); Assert.assertEquals("events written", 1000, hdfsWriter.getEventsWritten()); Assert.assertEquals("bytes written", 3000, hdfsWriter.getBytesWritten()); Assert.assertEquals("files opened", 10, hdfsWriter.getFilesOpened()); }
@Test public void testIntervalRoller() throws IOException, InterruptedException { final int ROLL_INTERVAL = 1; // seconds final int NUM_EVENTS = 10; final AtomicBoolean calledBack = new AtomicBoolean(false); MockHDFSWriter hdfsWriter = new MockHDFSWriter(); BucketWriter bucketWriter = new BucketWriter( ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, new HDFSEventSink.WriterCallback() { @Override public void run(String filePath) { calledBack.set(true); } }, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); Event e = EventBuilder.withBody("foo", Charsets.UTF_8); long startNanos = System.nanoTime(); for (int i = 0; i < NUM_EVENTS - 1; i++) { bucketWriter.append(e); } // sleep to force a roll... wait 2x interval just to be sure Thread.sleep(2 * ROLL_INTERVAL * 1000L); Assert.assertTrue(bucketWriter.closed); Assert.assertTrue(calledBack.get()); bucketWriter = new BucketWriter( ROLL_INTERVAL, 0, 0, 0, ctx, "/tmp", "file", "", ".tmp", null, null, SequenceFile.CompressionType.NONE, hdfsWriter, timedRollerPool, proxy, new SinkCounter("test-bucket-writer-" + System.currentTimeMillis()), 0, null, null, 30000, Executors.newSingleThreadExecutor(), 0, 0); // write one more event (to reopen a new file so we will roll again later) bucketWriter.append(e); long elapsedMillis = TimeUnit.MILLISECONDS.convert(System.nanoTime() - startNanos, TimeUnit.NANOSECONDS); long elapsedSeconds = elapsedMillis / 1000L; logger.info("Time elapsed: {} milliseconds", elapsedMillis); logger.info("Number of events written: {}", hdfsWriter.getEventsWritten()); logger.info("Number of bytes written: {}", hdfsWriter.getBytesWritten()); logger.info("Number of files opened: {}", hdfsWriter.getFilesOpened()); logger.info("Number of files closed: {}", hdfsWriter.getFilesClosed()); Assert.assertEquals("events written", NUM_EVENTS, hdfsWriter.getEventsWritten()); Assert.assertEquals( "bytes written", e.getBody().length * NUM_EVENTS, hdfsWriter.getBytesWritten()); Assert.assertEquals("files opened", 2, hdfsWriter.getFilesOpened()); // before auto-roll Assert.assertEquals("files closed", 1, hdfsWriter.getFilesClosed()); logger.info("Waiting for roll..."); Thread.sleep(2 * ROLL_INTERVAL * 1000L); logger.info("Number of files closed: {}", hdfsWriter.getFilesClosed()); Assert.assertEquals("files closed", 2, hdfsWriter.getFilesClosed()); }