protected void parseCommandLine(CommandLine cmdline) throws ParseException { if (!cmdline.hasOption("u")) { throw new ParseException("No distributedlog uri provided."); } uri = URI.create(cmdline.getOptionValue("u")); if (cmdline.hasOption("c")) { String configFile = cmdline.getOptionValue("c"); try { dlConf.loadConf(new File(configFile).toURI().toURL()); } catch (ConfigurationException e) { throw new ParseException("Failed to load distributedlog configuration from " + configFile + "."); } catch (MalformedURLException e) { throw new ParseException("Failed to load distributedlog configuration from " + configFile + ": malformed uri."); } } if (cmdline.hasOption("a")) { zkAclId = cmdline.getOptionValue("a"); } if (cmdline.hasOption("f")) { force = true; } }
protected void parseCommandLine(CommandLine cmdline) throws ParseException { if (!cmdline.hasOption("u")) { throw new ParseException("No distributedlog uri provided."); } String urisStr = cmdline.getOptionValue("u"); for (String uriStr : urisStr.split(",")) { uris.add(URI.create(uriStr)); } if (cmdline.hasOption("c")) { String configFile = cmdline.getOptionValue("c"); try { dlConf.loadConf(new File(configFile).toURI().toURL()); } catch (ConfigurationException e) { throw new ParseException("Failed to load distributedlog configuration from " + configFile + "."); } catch (MalformedURLException e) { throw new ParseException("Failed to load distributedlog configuration from malformed " + configFile + "."); } } if (cmdline.hasOption("a")) { zkAclId = cmdline.getOptionValue("a"); } if (cmdline.hasOption("f")) { force = true; } }
if (cmdline.hasOption("c")) { String configFile = cmdline.getOptionValue("c"); this.conf.loadConf(new File(configFile).toURI().toURL());
public TestAsyncReaderWriter() { this.testConf = new DistributedLogConfiguration(); this.testConf.loadConf(conf); this.testConf.setReaderIdleErrorThresholdMillis(1200000); }
@Test(timeout = 60000) public void testSimpleAsyncRead() throws Exception { String name = runtime.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(testConf); simpleAsyncReadTest(name, confLocal); }
@Test(timeout = 60000) public void testImmediateFlush() throws Exception { String name = "distrlog-immediate-flush"; DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); confLocal.setOutputBufferSize(0); testNonPartitionedWritesInternal(name, confLocal); }
@Test(timeout = 60000) public void testSimpleAsyncReadWriteWithMonitoredFuturePool() throws Exception { String name = runtime.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(testConf); confLocal.setTaskExecutionWarnTimeMicros(1000); confLocal.setEnableTaskExecutionStats(true); simpleAsyncReadTest(name, confLocal); }
@Test(timeout = 60000) public void testSkipToSkipsBytesWithImmediateFlush() throws Exception { String name = testNames.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); confLocal.setImmediateFlushEnabled(true); confLocal.setOutputBufferSize(0); skipForwardThenSkipBack(name, confLocal); }
@Test(timeout = 60000) public void testSkipToSkipsBytesWithLargerLogRecords() throws Exception { String name = testNames.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); confLocal.setImmediateFlushEnabled(false); confLocal.setOutputBufferSize(1024*100); confLocal.setPeriodicFlushFrequencyMilliSeconds(1000*60); skipForwardThenSkipBack(name, confLocal); }
@Test public void testExplicitTruncation() throws Exception { String name = "distrlog-truncation-explicit"; DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); confLocal.setExplicitTruncationByApplication(true); Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>(); Pair<DistributedLogManager, AsyncLogWriter> pair = populateData(txid2DLSN, confLocal, name, 4, 10, true); Thread.sleep(1000); for (int i = 1; i <= 4; i++) { int txn = (i-1) * 10 + i; DLSN dlsn = txid2DLSN.get((long)txn); assertTrue(Await.result(pair.getRight().truncate(dlsn))); verifyEntries(name, 1, (i - 1) * 10 + 1, (5 - i + 1) * 10); } // Delete higher dlsn int txn = 43; DLSN dlsn = txid2DLSN.get((long) txn); assertTrue(Await.result(pair.getRight().truncate(dlsn))); verifyEntries(name, 1, 31, 20); Utils.close(pair.getRight()); pair.getLeft().close(); // Try force truncation BKDistributedLogManager dlm = (BKDistributedLogManager)createNewDLM(confLocal, name); BKLogWriteHandler handler = dlm.createWriteHandler(true); FutureUtils.result(handler.purgeLogSegmentsOlderThanTxnId(Integer.MAX_VALUE)); verifyEntries(name, 1, 31, 20); }
@Test(timeout = 60000) public void testMaxTransmissionSize() throws Exception { DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); confLocal.setOutputBufferSize(1024 * 1024); DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(confLocal, "distrlog-transmissionSize"); long txid = 1; BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1); boolean exceptionEncountered = false; byte[] largePayload = DLMTestUtil.repeatString(DLMTestUtil.repeatString("abcdefgh", 256), 256).getBytes(); try { while (txid < 3) { LogRecord op = new LogRecord(txid, largePayload); out.write(op); txid++; } } catch (LogRecordTooLongException exc) { exceptionEncountered = true; } finally { FutureUtils.result(out.asyncClose()); } bkdlmAndClients.close(); assert(!exceptionEncountered); }
String name = "distrlog-close-and-complete-logsegment-when-close-failed"; DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(testConf); confLocal.setOutputBufferSize(0); confLocal.setImmediateFlushEnabled(true);
@Test(timeout = 60000) public void testGetFirstDLSNWithOpenLedger() throws Exception { String dlName = runtime.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); confLocal.setImmediateFlushEnabled(true); confLocal.setOutputBufferSize(0); int numEntriesPerSegment = 100; DistributedLogManager dlm1 = createNewDLM(confLocal, dlName); long txid = 1; ArrayList<Future<DLSN>> futures = new ArrayList<Future<DLSN>>(numEntriesPerSegment); AsyncLogWriter out = dlm1.startAsyncLogSegmentNonPartitioned(); for (int eid = 0; eid < numEntriesPerSegment; ++eid) { futures.add(out.write(DLMTestUtil.getLogRecordInstance(txid))); ++txid; } for (Future<DLSN> future : futures) { Await.result(future); } BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm1).createReadHandler(); DLSN last = dlm1.getLastDLSN(); assertEquals(new DLSN(1,99,0), last); DLSN first = Await.result(dlm1.getFirstDLSNAsync()); assertEquals(new DLSN(1,0,0), first); Utils.close(out); }
@Test(timeout = 60000) public void testBulkAsyncReadWithWriteBatch() throws Exception { String name = "distrlog-bulkasyncread-with-writebatch"; DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(conf); confLocal.setOutputBufferSize(1024000); confLocal.setReadAheadWaitTime(10); confLocal.setReadAheadMaxRecords(10000); confLocal.setReadAheadBatchSize(10); DistributedLogManager dlm = createNewDLM(confLocal, name); int numLogSegments = 3; int numRecordsPerLogSegment = 20; writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, 1L, false); final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN); int expectedTxID = 1; for (long i = 0; i < 3; i++) { // since we batched 20 entries into single bookkeeper entry // we should be able to read 20 entries as a batch. List<LogRecordWithDLSN> records = Await.result(reader.readBulk(20)); assertEquals(20, records.size()); for (LogRecordWithDLSN record : records) { assertEquals(expectedTxID, record.getTransactionId()); ++expectedTxID; } } Utils.close(reader); dlm.close(); }
/** * Test Case: A large write batch will span multiple packets. * @throws Exception */ @Test(timeout = 60000) public void testAsyncBulkWriteSpanningPackets() throws Exception { String name = "distrlog-testAsyncBulkWriteSpanningPackets"; DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(testConf); confLocal.setOutputBufferSize(1024); DistributedLogManager dlm = createNewDLM(confLocal, name); BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned()); // First entry. int numTransmissions = 4; int recSize = 10*1024; int batchSize = (numTransmissions*MAX_LOGRECORDSET_SIZE+1)/recSize; long ledgerIndex = 1; long entryIndex = 0; long slotIndex = 0; long txIndex = 1; DLSN dlsn = checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex); assertEquals(4, dlsn.getEntryId()); assertEquals(1, dlsn.getLogSegmentSequenceNo()); writer.closeAndComplete(); dlm.close(); }
@Test(timeout = 60000) public void testReadBrokenEntries() throws Exception { String name = runtime.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(testConf); confLocal.setOutputBufferSize(0); confLocal.setPeriodicFlushFrequencyMilliSeconds(0); confLocal.setImmediateFlushEnabled(true); confLocal.setReadAheadWaitTime(10); confLocal.setReadAheadBatchSize(1); confLocal.setPositionGapDetectionEnabled(false); confLocal.setReadAheadSkipBrokenEntries(true); confLocal.setEIInjectReadAheadBrokenEntries(true); DistributedLogManager dlm = createNewDLM(confLocal, name); int numLogSegments = 3; int numRecordsPerLogSegment = 10; long txid = 1L; txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false); AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN); // 3 segments, 10 records each, immediate flush, batch size 1, so just the first // record in each ledger is discarded, for 30 - 3 = 27 records. for (int i = 0; i < 27; i++) { LogRecordWithDLSN record = Await.result(reader.readNext()); assertFalse(record.getDlsn().getEntryId() % 10 == 0); } Utils.close(reader); dlm.close(); }
String name = runtime.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(testConf);
@Test(timeout = 60000) public void testReadBrokenEntriesAndLargeBatchSizeCrossSegment() throws Exception { String name = runtime.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(testConf); confLocal.setOutputBufferSize(0); confLocal.setPeriodicFlushFrequencyMilliSeconds(0); confLocal.setImmediateFlushEnabled(true); confLocal.setReadAheadWaitTime(10); confLocal.setReadAheadBatchSize(8); confLocal.setPositionGapDetectionEnabled(false); confLocal.setReadAheadSkipBrokenEntries(true); confLocal.setEIInjectReadAheadBrokenEntries(true); DistributedLogManager dlm = createNewDLM(confLocal, name); int numLogSegments = 3; int numRecordsPerLogSegment = 5; long txid = 1L; txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false); AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN); // Every 10th record broken. Reading 8 at once, beginning from 0: // 1. range 0-7 will be corrupted and discarded // 2. range 1-8 will be good, but only contain 4 records // And so on for the next segment, so 4 records in each segment, for 12 good records for (int i = 0; i < 12; i++) { LogRecordWithDLSN record = Await.result(reader.readNext()); assertFalse(record.getDlsn().getEntryId() % 10 == 0); } Utils.close(reader); dlm.close(); }
String name = runtime.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(testConf);
@Test(timeout = 60000) public void testAsyncWritePendingWritesAbortedWhenLedgerRollTriggerFails() throws Exception { String name = runtime.getMethodName(); DistributedLogConfiguration confLocal = new DistributedLogConfiguration(); confLocal.loadConf(testConf); confLocal.setOutputBufferSize(1024); confLocal.setMaxLogSegmentBytes(1024); confLocal.setLogSegmentRollingIntervalMinutes(0); DistributedLogManager dlm = createNewDLM(confLocal, name); BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned()); // Write one record larger than max seg size. Ledger doesn't roll until next write. int txid = 1; LogRecord record = DLMTestUtil.getLogRecordInstance(txid++, 2048); Future<DLSN> result = writer.write(record); DLSN dlsn = Await.result(result, Duration.fromSeconds(10)); assertEquals(1, dlsn.getLogSegmentSequenceNo()); record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1); result = writer.write(record); validateFutureFailed(result, LogRecordTooLongException.class); record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1); result = writer.write(record); validateFutureFailed(result, WriteException.class); record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1); result = writer.write(record); validateFutureFailed(result, WriteException.class); writer.closeAndComplete(); dlm.close(); }