@Override public Future<Void> asyncAbort() { return asyncClose(); }
@VisibleForTesting LogSegmentMetadata completeAndCloseLogSegment(long logSegmentSeqNo, long ledgerId, long firstTxId, long lastTxId, int recordCount) throws IOException { return completeAndCloseLogSegment(inprogressZNodeName(ledgerId, firstTxId, logSegmentSeqNo), logSegmentSeqNo, ledgerId, firstTxId, lastTxId, recordCount, -1, -1); }
List<LogSegmentMetadata> ledgerListDesc = getFilteredLedgerListDesc(false, false); Long nextLogSegmentSeqNo = DLUtils.nextLogSegmentSequenceNumber(ledgerListDesc); logSegmentSeqNo, getFullyQualifiedName()); } else if (maxLogSegmentSequenceNo.getSequenceNumber() + 1 != logSegmentSeqNo) { LOG.warn("Unexpected max log segment sequence number {} for {} : list of cached segments = {}", new Object[]{maxLogSegmentSequenceNo.getSequenceNumber(), getFullyQualifiedName(), getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)}); + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName() + ", expected " + (logSegmentSeqNo - 1));
@Override public Future<LogSegmentMetadata> apply(final LogSegmentMetadata l) { if (!l.isInProgress()) { return Future.value(l); } LOG.info("Recovering last record in log segment {} for {}.", l, getFullyQualifiedName()); return asyncReadLastRecord(l, true, true, true).flatMap( new AbstractFunction1<LogRecordWithDLSN, Future<LogSegmentMetadata>>() { @Override public Future<LogSegmentMetadata> apply(LogRecordWithDLSN lastRecord) { return completeLogSegment(l, lastRecord); } }); }
private Future<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(List<LogSegmentMetadata> logSegments, final DLSN dlsn) { LOG.debug("Setting truncation status on logs older than {} from {} for {}", new Object[]{dlsn, logSegments, getFullyQualifiedName()}); List<LogSegmentMetadata> truncateList = new ArrayList<LogSegmentMetadata>(logSegments.size()); LogSegmentMetadata partialTruncate = null; LOG.info("{}: Truncating log segments older than {}", getFullyQualifiedName(), dlsn); int numCandidates = getNumCandidateLogSegmentsToTruncate(logSegments); for (int i = 0; i < numCandidates; i++) { LogSegmentMetadata l = logSegments.get(i); if (!l.isInProgress()) { if (l.getLastDLSN().compareTo(dlsn) < 0) { LOG.debug("{}: Truncating log segment {} ", getFullyQualifiedName(), l); truncateList.add(l); } else if (l.getFirstDLSN().compareTo(dlsn) < 0) { String logMsg = String.format("Potential metadata inconsistency for stream %s at segment %s", getFullyQualifiedName(), l); LOG.error(logMsg); return Future.exception(new DLIllegalStateException(logMsg)); LOG.info("{}: Partially truncating log segment {} older than {}.", new Object[] {getFullyQualifiedName(), l, dlsn}); partialTruncate = l; } else { return setLogSegmentTruncationStatus(truncateList, partialTruncate, dlsn);
BKDistributedLogManager dlm = (BKDistributedLogManager) manager; BKLogWriteHandler writeHandler = dlm.createWriteHandler(false); FutureUtils.result(writeHandler.lockHandler()); String inprogressZnodeName = writeHandler.inprogressZNodeName(lh.getId(), startTxID, logSegmentSeqNo); String znodePath = writeHandler.inprogressZNode(lh.getId(), startTxID, logSegmentSeqNo); LogSegmentMetadata l = new LogSegmentMetadata.LogSegmentMetadataBuilder(znodePath, l.write(dlm.writerZKC); writeHandler.maxTxId.store(startTxID); writeHandler.addLogSegmentToCache(inprogressZnodeName, l); BKLogSegmentWriter writer = new BKLogSegmentWriter( writeHandler.getFullyQualifiedName(), inprogressZnodeName, conf, FutureUtils.result(writeHandler.completeAndCloseLogSegment(writer)); FutureUtils.result(writeHandler.unlockHandler());
long lastSlotId) throws IOException { Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>(); doCompleteAndCloseLogSegment( inprogressZnodeName, logSegmentSeqNo, final String inprogressZnodePath = inprogressZNode(inprogressZnodeName); LogSegmentMetadata inprogressLogSegment = readLogSegmentFromCache(inprogressZnodeName); } else { LOG.warn("Unexpected max ledger sequence number {} found while completing log segment {} for {}", new Object[] { maxLogSegmentSequenceNo.getSequenceNumber(), logSegmentSeqNo, getFullyQualifiedName() }); if (validateLogSegmentSequenceNumber) { FutureUtils.setException(promise, new DLIllegalStateException("Unexpected max log segment sequence number " + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName() + ", expected " + (logSegmentSeqNo - 1))); return; final String nameForCompletedLedger = completedLedgerZNodeName(firstTxId, lastTxId, logSegmentSeqNo); final String pathForCompletedLedger = completedLedgerZNode(firstTxId, lastTxId, logSegmentSeqNo); long startSequenceId; try { startSequenceId = computeStartSequenceId(inprogressLogSegment); } catch (IOException ioe) { FutureUtils.setException(promise, ioe); lastSlotId, startSequenceId); setLastLedgerRollingTimeMillis(completedLogSegment.getCompletionTime());
@Test(timeout = 60000) public void testRecoveryEmptyLedger() throws Exception { DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, "distrlog-recovery-empty-ledger"); BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1); long txid = 1; for (long i = 1; i <= 100; i++) { LogRecord op = DLMTestUtil.getLogRecordInstance(txid++); out.write(op); if ((i % 10) == 0) { FutureUtils.result(out.flushAndCommit()); } } FutureUtils.result(out.flushAndCommit()); FutureUtils.result(out.asyncClose()); bkdlmAndClients.getWriteHandler().completeAndCloseLogSegment(out.getLogSegmentSequenceNumber(), out.getLogSegmentId(), 1, 100, 100); assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(1, 100, out.getLogSegmentSequenceNumber()), false)); BKLogSegmentWriter outEmpty = bkdlmAndClients.getWriteHandler().startLogSegment(101); Abortables.abort(outEmpty, false); assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(101, 101, outEmpty.getLogSegmentSequenceNumber()), false)); assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(outEmpty.getLogSegmentId(), 101, outEmpty.getLogSegmentSequenceNumber()), false)); FutureUtils.result(bkdlmAndClients.getWriteHandler().recoverIncompleteLogSegments()); assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(outEmpty.getLogSegmentId(), outEmpty.getLogSegmentSequenceNumber(), 101), false)); assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(101, 101, outEmpty.getLogSegmentSequenceNumber()), false)); }
zkc.exists(blplm.completedLedgerZNode(start, txid - 1, perStreamLogWriter.getLogSegmentSequenceNumber()), false)); BKLogSegmentWriter writer = blplm.startLogSegment(txid - 1); blplm.completeAndCloseLogSegment(writer.getLogSegmentSequenceNumber(), writer.getLogSegmentId(), txid - 1, txid - 1, 0); assertNotNull( zkc.exists(blplm.completedLedgerZNode(txid - 1, txid - 1, writer.getLogSegmentSequenceNumber()), false)); FutureUtils.result(blplm.asyncClose());
FailpointUtils.checkFailPoint( FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber); logSegmentSeqNo = assignLogSegmentSequenceNumber(); } catch (IOException e) { failStartLogSegment(promise, bestEffort, e); return; final String inprogressZnodeName = inprogressZNodeName(lh.getId(), txId, logSegmentSeqNo); final String inprogressZnodePath = inprogressZNode(lh.getId(), txId, logSegmentSeqNo); final LogSegmentMetadata l = new LogSegmentMetadata.LogSegmentMetadataBuilder(inprogressZnodePath, writeLogSegment( txn, zooKeeperClient.getDefaultACL(), storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, logSegmentSeqNo, true); storeMaxTxId(txn, maxTxId, txId);
private long writeAndMarkEndOfStream(DistributedLogManager dlm, long txid) throws Exception { for (long i = 0; i < 3; i++) { long start = txid; BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned(); for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) { writer.write(DLMTestUtil.getLogRecordInstance(txid++)); } BKLogSegmentWriter perStreamLogWriter = writer.getCachedLogWriter(); if (i < 2) { writer.closeAndComplete(); BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true); assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1, perStreamLogWriter.getLogSegmentSequenceNumber()), false)); FutureUtils.result(blplm.asyncClose()); } else { writer.markEndOfStream(); BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true); assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, DistributedLogConstants.MAX_TXID, perStreamLogWriter.getLogSegmentSequenceNumber()), false)); FutureUtils.result(blplm.asyncClose()); } } return txid; }
@Test(timeout = 60000) public void testGetFilteredLedgerListInWriteHandler() throws Exception { String dlName = runtime.getMethodName(); prepareLogSegments(dlName, 11, 3); BKDistributedLogManager dlm = createNewDLM(conf, dlName); // Get full list. BKLogWriteHandler writeHandler0 = dlm.createWriteHandler(false); List<LogSegmentMetadata> cachedFullLedgerList = writeHandler0.getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR); assertTrue(cachedFullLedgerList.size() <= 1); List<LogSegmentMetadata> fullLedgerList = writeHandler0.getFullLedgerListDesc(false, false); assertEquals(11, fullLedgerList.size()); // Get filtered list. BKLogWriteHandler writeHandler1 = dlm.createWriteHandler(false); List<LogSegmentMetadata> filteredLedgerListDesc = writeHandler1.getFilteredLedgerListDesc(false, false); assertEquals(1, filteredLedgerListDesc.size()); assertEquals(fullLedgerList.get(0), filteredLedgerListDesc.get(0)); List<LogSegmentMetadata> filteredLedgerList = writeHandler1.getFilteredLedgerList(false, false); assertEquals(1, filteredLedgerList.size()); assertEquals(fullLedgerList.get(0), filteredLedgerList.get(0)); }
assertNull(zkc.exists(blplm1.completedLedgerZNode(1, 100, perStreamLogWriter.getLogSegmentSequenceNumber()), false)); assertNotNull(zkc.exists(blplm1.inprogressZNode(perStreamLogWriter.getLogSegmentId(), 1, perStreamLogWriter.getLogSegmentSequenceNumber()), false)); assertNotNull(zkc.exists(blplm1.completedLedgerZNode(1, 100, perStreamLogWriter.getLogSegmentSequenceNumber()), false)); assertNull(zkc.exists(blplm1.inprogressZNode(perStreamLogWriter.getLogSegmentId(), 1, perStreamLogWriter.getLogSegmentSequenceNumber()), false)); FutureUtils.result(blplm1.asyncClose()); assertEquals(100, dlm.getLogRecordCount()); dlm.close();
/** * The DistributedLogManager may archive/purge any logs for transactionId * less than or equal to minImageTxId. * This is to be used only when the client explicitly manages deletion. If * the cleanup policy is based on sliding time window, then this method need * not be called. * * @param minTxIdToKeep the earliest txid that must be retained * @throws IOException if purging fails */ @Override public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException { Preconditions.checkArgument(minTxIdToKeep > 0, "Invalid transaction id " + minTxIdToKeep); checkClosedOrInError("purgeLogSegmentsOlderThan"); BKLogWriteHandler ledgerHandler = createWriteHandler(true); try { LOG.info("Purging logs for {} older than {}", ledgerHandler.getFullyQualifiedName(), minTxIdToKeep); FutureUtils.result(ledgerHandler.purgeLogSegmentsOlderThanTxnId(minTxIdToKeep)); } finally { Utils.closeQuietly(ledgerHandler); } }
final BKLogWriteHandler writeHandler = new BKLogWriteHandler( logMetadata, conf, PermitManager manager = getLogSegmentRollingPermitManager(); if (manager instanceof Watcher) { writeHandler.register((Watcher) manager); writeHandler.lockHandler().addEventListener(new FutureEventListener<DistributedLock>() { @Override public void onSuccess(DistributedLock lock) {
private void prepareLogSegments(String name, int numSegments, int numEntriesPerSegment) throws Exception { DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, name); long txid = 1; for (int sid = 0; sid < numSegments; ++sid) { BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(txid); for (int eid = 0; eid < numEntriesPerSegment; ++eid) { LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid); out.write(record); ++txid; } FutureUtils.result(out.asyncClose()); bkdlmAndClients.getWriteHandler().completeAndCloseLogSegment( out.getLogSegmentSequenceNumber(), out.getLogSegmentId(), 1 + sid * numEntriesPerSegment, (sid + 1) * numEntriesPerSegment, numEntriesPerSegment); } bkdlmAndClients.close(); }
static void writeRecordsForNonBlockingReads(DistributedLogConfiguration conf, DistributedLogManager dlm, boolean recover, long segmentSize) throws Exception { long txId = 1; for (long i = 0; i < 3; i++) { BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned(); for (long j = 1; j < segmentSize; j++) { FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++))); } if (recover) { FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++))); TimeUnit.MILLISECONDS.sleep(300); writer.abort(); LOG.debug("Recovering Segments"); BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true); FutureUtils.result(blplm.recoverIncompleteLogSegments()); FutureUtils.result(blplm.asyncClose()); LOG.debug("Recovered Segments"); } else { FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++))); writer.closeAndComplete(); } TimeUnit.MILLISECONDS.sleep(300); } }
private Future<List<LogSegmentMetadata>> deleteLogSegments( final List<LogSegmentMetadata> logs) { if (LOG.isTraceEnabled()) { LOG.trace("Purging logs for {} : {}", getFullyQualifiedName(), logs); } return FutureUtils.processList(logs, new Function<LogSegmentMetadata, Future<LogSegmentMetadata>>() { @Override public Future<LogSegmentMetadata> apply(LogSegmentMetadata segment) { return deleteLogSegment(segment); } }, scheduler); }
/** * Finalize a log segment. If the journal manager is currently * writing to a ledger, ensure that this is the ledger of the log segment * being finalized. * <p/> * Otherwise this is the recovery case. In the recovery case, ensure that * the firstTxId of the ledger matches firstTxId for the segment we are * trying to finalize. */ Future<LogSegmentMetadata> completeAndCloseLogSegment(final BKLogSegmentWriter writer) { final Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>(); completeAndCloseLogSegment(writer, promise); return promise; }
Future<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTimestamp(final long minTimestampToKeep) { if (minTimestampToKeep >= Utils.nowInMillis()) { return Future.exception(new IllegalArgumentException( "Invalid timestamp " + minTimestampToKeep + " to purge logs for " + getFullyQualifiedName())); } return asyncGetFullLedgerList(false, false).flatMap( new Function<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() { @Override public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) { List<LogSegmentMetadata> purgeList = new ArrayList<LogSegmentMetadata>(logSegments.size()); int numCandidates = getNumCandidateLogSegmentsToTruncate(logSegments); for (int iterator = 0; iterator < numCandidates; iterator++) { LogSegmentMetadata l = logSegments.get(iterator); // When application explicitly truncates segments; timestamp based purge is // only used to cleanup log segments that have been marked for truncation if ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) && !l.isInProgress() && (l.getCompletionTime() < minTimestampToKeep)) { purgeList.add(l); } else { // stop truncating log segments if we find either an inprogress or a partially // truncated log segment break; } } LOG.info("Deleting log segments older than {} for {} : {}", new Object[] { minTimestampToKeep, getFullyQualifiedName(), purgeList }); return deleteLogSegments(purgeList); } }); }