@Override public long getLastTxId() throws IOException { checkClosedOrInError("getLastTxId"); return FutureUtils.result(getLastTxIdAsync()); }
/** * Get the last log record in the stream * * @return the last log record in the stream * @throws java.io.IOException if a stream cannot be found. */ @Override public LogRecordWithDLSN getLastLogRecord() throws IOException { checkClosedOrInError("getLastLogRecord"); return FutureUtils.result(getLastLogRecordAsync()); }
/** * Get the number of log records in the active portion of the log * Any log segments that have already been truncated will not be included * * @return number of log records * @throws IOException */ @Override public long getLogRecordCount() throws IOException { checkClosedOrInError("getLogRecordCount"); return FutureUtils.result(getLogRecordCountAsync(DLSN.InitialDLSN)); }
@Override public long getFirstTxId() throws IOException { checkClosedOrInError("getFirstTxId"); return FutureUtils.result(getFirstRecordAsyncInternal()).getTransactionId(); }
/** * Check if an end of stream marker was added to the stream * A stream with an end of stream marker cannot be appended to * * @return true if the marker was added to the stream, false otherwise */ @Override public boolean isEndOfStreamMarked() throws IOException { checkClosedOrInError("isEndOfStreamMarked"); long lastTxId = FutureUtils.result(getLastLogRecordAsyncInternal(false, true)).getTransactionId(); return lastTxId == DistributedLogConstants.MAX_TXID; }
@Override public DLSN getLastDLSN() throws IOException { checkClosedOrInError("getLastDLSN"); return FutureUtils.result(getLastLogRecordAsyncInternal(false, false)).getDlsn(); }
private boolean checkClosedOrInError(String operation) { if (null == lastException.get()) { try { if (null != bkLedgerManager && null != bkLedgerManager.readAheadWorker) { bkLedgerManager.readAheadWorker.checkClosedOrInError(); } bkDistributedLogManager.checkClosedOrInError(operation); } catch (IOException exc) { setLastException(exc); } } if (lockStream) { try { bkLedgerManager.checkReadLock(); } catch (IOException ex) { setLastException(ex); } } if (null != lastException.get()) { LOG.trace("Cancelling pending reads"); cancelAllPendingReads(lastException.get()); return true; } return false; }
LogReader getInputStreamInternal(DLSN fromDLSN, Optional<Long> fromTxnId) throws IOException { LOG.info("Create async reader starting from {}", fromDLSN); checkClosedOrInError("getInputStream"); Optional<String> subscriberId = Optional.absent(); BKAsyncLogReaderDLSN asyncReader = new BKAsyncLogReaderDLSN( this, scheduler, getLockStateExecutor(true), fromDLSN, subscriberId, true, dynConf.getDeserializeRecordSetOnReads(), statsLogger); pendingReaders.add(asyncReader); return new BKSyncLogReaderDLSN(conf, asyncReader, scheduler, fromTxnId); }
/** * 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); } }
/** * Recover a specified stream within the log container * The writer implicitly recovers a topic when it resumes writing. * This allows applications to recover a container explicitly so * that application may read a fully recovered log before resuming * the writes * * @throws IOException if the recovery fails */ private void recoverInternal(String streamIdentifier) throws IOException { checkClosedOrInError("recoverInternal"); BKLogWriteHandler ledgerHandler = createWriteHandler(true); try { FutureUtils.result(ledgerHandler.recoverIncompleteLogSegments()); } finally { Utils.closeQuietly(ledgerHandler); } }
@Override public Future<AsyncLogWriter> openAsyncLogWriter() { try { checkClosedOrInError("startLogSegmentNonPartitioned"); } catch (AlreadyClosedException e) { return Future.exception(e);
/** * Begin writing to the log stream identified by the name * * @return the writer interface to generate log records */ @Override public BKSyncLogWriter startLogSegmentNonPartitioned() throws IOException { checkClosedOrInError("startLogSegmentNonPartitioned"); BKSyncLogWriter writer = new BKSyncLogWriter(conf, dynConf, this); boolean success = false; try { writer.createAndCacheWriteHandler(); BKLogWriteHandler writeHandler = writer.getWriteHandler(); FutureUtils.result(writeHandler.lockHandler()); success = true; return writer; } finally { if (!success) { writer.abort(); } } }