ISSUE #165: Enable checkstyle in core module
Descriptions of the changes in this PR:
Fix checkstyle error in core module
Author: Arvin <arvindevel@gmail.com>
Reviewers: Sijie Guo <sijie@apache.org>
This closes #192 from ArvinDevel/enable_check, closes #165
diff --git a/distributedlog-build-tools/src/main/resources/distributedlog/checkstyle.xml b/distributedlog-build-tools/src/main/resources/distributedlog/checkstyle.xml
index 342f421..c68e7c0 100644
--- a/distributedlog-build-tools/src/main/resources/distributedlog/checkstyle.xml
+++ b/distributedlog-build-tools/src/main/resources/distributedlog/checkstyle.xml
@@ -100,6 +100,7 @@
<module name="AvoidStarImport">
<property name="severity" value="error"/>
+ <property name="allowStaticMemberImports" value="true"/>
</module>
<module name="IllegalImport">
@@ -216,7 +217,7 @@
<property name="applyToProtected" value="true"/>
<property name="applyToPackage" value="true"/>
<property name="applyToPrivate" value="true"/>
- <property name="format" value="^[a-z][a-zA-Z0-9]*_?$"/>
+ <property name="format" value="^[a-zA-Z0-9_]*$"/>
<property name="severity" value="error"/>
</module>
@@ -227,7 +228,7 @@
<property name="applyToProtected" value="true"/>
<property name="applyToPackage" value="true"/>
<property name="applyToPrivate" value="true"/>
- <property name="format" value="^[a-z][a-zA-Z0-9]*$"/>
+ <property name="format" value="^[a-zA-Z0-9_]*$"/>
<property name="severity" value="error"/>
</module>
@@ -246,13 +247,15 @@
<module name="LocalFinalVariableName">
<!-- Validates identifiers for local final variables against the
- expression "^[a-z][a-zA-Z0-9]*$". -->
+ expression "^[a-z][a-zA-Z0-9_]*$". -->
+ <property name="format" value="^[a-z][a-zA-Z0-9_]*$"/>
<property name="severity" value="error"/>
</module>
<module name="LocalVariableName">
<!-- Validates identifiers for local variables against the
- expression "^[a-z][a-zA-Z0-9]*$". -->
+ expression "^[a-z][a-zA-Z0-9_]*$". -->
+ <property name="format" value="^[a-z][a-zA-Z0-9_]*$"/>
<property name="severity" value="error"/>
</module>
diff --git a/distributedlog-build-tools/src/main/resources/distributedlog/suppressions-core.xml b/distributedlog-build-tools/src/main/resources/distributedlog/suppressions-core.xml
deleted file mode 100644
index dd7eef0..0000000
--- a/distributedlog-build-tools/src/main/resources/distributedlog/suppressions-core.xml
+++ /dev/null
@@ -1,55 +0,0 @@
-<?xml version="1.0"?>
-<!--
- Licensed under the Apache License, Version 2.0 (the "License");
- you may not use this file except in compliance with the License.
- You may obtain a copy of the License at
-
- http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License. See accompanying LICENSE file.
--->
-<!DOCTYPE suppressions PUBLIC
-"-//Puppy Crawl//DTD Suppressions 1.1//EN"
-"http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
-
-<suppressions>
- <suppress checks="JavadocPackage" files=".*[\\/]src[\\/]test[\\/].*"/>
- <suppress checks="JavadocPackage" files=".*[\\/]maven-archetypes[\\/].*"/>
- <suppress checks="JavadocPackage" files=".*[\\/]examples[\\/].*"/>
-
- <!-- suppress packages by packages -->
- <!-- TODO: enable checkstyle package by package in https://github.com/apache/distributedlog/issues/165 -->
- <suppress checks=".*" files=".*[\\/]distributedlog[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.acl[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.admin[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.auditor[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.bk[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.callback[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.config[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.exceptions[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.feature[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.function[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.impl[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.injector[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.limiter[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.lock[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.logsegment[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.metadata[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.namespace[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.net[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.selector[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.tools[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.util[\\/].*"/>
- <suppress checks=".*" files=".*[\\/]distributedlog\.zk[\\/].*"/>
-
- <!-- suppress all checks in the generated directories -->
- <suppress checks=".*" files=".*[\\/]distributedlog\.thrift[\\/].*"/>
- <suppress checks=".*" files=".+[\\/]generated[\\/].+\.java" />
- <suppress checks=".*" files=".+[\\/]generated-sources[\\/].+\.java" />
- <suppress checks=".*" files=".+[\\/]generated-test-sources[\\/].+\.java" />
-</suppressions>
-
diff --git a/distributedlog-build-tools/src/main/resources/distributedlog/suppressions.xml b/distributedlog-build-tools/src/main/resources/distributedlog/suppressions.xml
index 9f60c25..fdea0b1 100644
--- a/distributedlog-build-tools/src/main/resources/distributedlog/suppressions.xml
+++ b/distributedlog-build-tools/src/main/resources/distributedlog/suppressions.xml
@@ -20,8 +20,9 @@
<suppress checks="JavadocPackage" files=".*[\\/]src[\\/]test[\\/].*"/>
<suppress checks="JavadocPackage" files=".*[\\/]maven-archetypes[\\/].*"/>
<suppress checks="JavadocPackage" files=".*[\\/]examples[\\/].*"/>
-
+
<!-- suppress all checks in the generated directories -->
+ <suppress checks=".*" files=".*[\\/]distributedlog\.thrift[\\/].*"/>
<suppress checks=".*" files=".+[\\/]generated[\\/].+\.java" />
<suppress checks=".*" files=".+[\\/]generated-sources[\\/].+\.java" />
<suppress checks=".*" files=".+[\\/]generated-test-sources[\\/].+\.java" />
diff --git a/distributedlog-core/pom.xml b/distributedlog-core/pom.xml
index 14fa60f..5baf775 100644
--- a/distributedlog-core/pom.xml
+++ b/distributedlog-core/pom.xml
@@ -350,15 +350,16 @@
</dependencies>
<configuration>
<configLocation>distributedlog/checkstyle.xml</configLocation>
- <suppressionsLocation>distributedlog/suppressions-core.xml</suppressionsLocation>
+ <suppressionsLocation>distributedlog/suppressions.xml</suppressionsLocation>
<consoleOutput>true</consoleOutput>
<failOnViolation>true</failOnViolation>
<includeResources>false</includeResources>
<includeTestSourceDirectory>true</includeTestSourceDirectory>
+ <excludes>**/thrift/**/*</excludes>
</configuration>
<executions>
<execution>
- <phase>validate</phase>
+ <phase>test-compile</phase>
<goals>
<goal>check</goal>
</goals>
diff --git a/distributedlog-core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java b/distributedlog-core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
index b102912..63c18d4 100644
--- a/distributedlog-core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
+++ b/distributedlog-core/src/main/java/org/apache/bookkeeper/client/LedgerReader.java
@@ -105,6 +105,7 @@
} catch (BKException.BKDigestMatchException e) {
rr = new ReadResult<>(
eid, BKException.Code.DigestMatchException, null, bookieAddress.getSocketAddress());
+
} finally {
buffer.release();
}
@@ -159,6 +160,7 @@
return;
}
+
if (LedgerHandle.INVALID_ENTRY_ID >= recoveryData.lastAddConfirmed) {
callback.operationComplete(BKException.Code.OK, resultList);
return;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
index 3a31907..8bc2ab6 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
@@ -17,14 +17,18 @@
*/
package org.apache.distributedlog;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkNotNull;
import java.io.IOException;
import java.io.InputStream;
+
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+/**
+ * AppendOnlyStreamReader.
+ */
public class AppendOnlyStreamReader extends InputStream {
static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamReader.class);
@@ -40,7 +44,7 @@
private final LogRecordWithDLSN logRecord;
LogRecordWithInputStream(LogRecordWithDLSN logRecord) {
- Preconditions.checkNotNull(logRecord);
+ checkNotNull(logRecord);
LOG.debug("Got record dlsn = {}, txid = {}, len = {}",
new Object[] {logRecord.getDlsn(), logRecord.getTransactionId(), logRecord.getPayload().length});
@@ -65,7 +69,7 @@
}
/**
- * Construct ledger input stream
+ * Construct ledger input stream.
*
* @param dlm the Distributed Log Manager to access the stream
*/
@@ -183,7 +187,7 @@
byte[] skipBuffer = new byte[SKIP_BUFFER_SIZE];
while (currentPosition < position) {
long bytesToRead = Math.min(position - currentPosition, SKIP_BUFFER_SIZE);
- long bytesRead = read(skipBuffer, 0, (int)bytesToRead);
+ long bytesRead = read(skipBuffer, 0, (int) bytesToRead);
if (bytesRead < bytesToRead) {
return false;
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
index dde78c2..93ad610 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
@@ -20,12 +20,15 @@
import java.io.Closeable;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.exceptions.UnexpectedException;
import org.apache.distributedlog.common.concurrent.FutureEventListener;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.UnexpectedException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+/**
+ * AppendOnlyStreamWriter.
+ */
public class AppendOnlyStreamWriter implements Closeable {
static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamWriter.class);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
index 367bb50..9746d56 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
@@ -19,14 +19,14 @@
interface AsyncNotification {
/**
- * Triggered when the background activity encounters an exception
+ * Triggered when the background activity encounters an exception.
*
* @param reason the exception that encountered.
*/
void notifyOnError(Throwable reason);
/**
- * Triggered when the background activity completes an operation
+ * Triggered when the background activity completes an operation.
*/
void notifyOnOperationComplete();
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
index 8a0bffb..1e24225 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
@@ -18,9 +18,17 @@
package org.apache.distributedlog;
import com.google.common.annotations.VisibleForTesting;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
+import java.util.concurrent.TimeUnit;
import java.util.function.Function;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.PermitManager;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.exceptions.AlreadyClosedException;
import org.apache.distributedlog.exceptions.LockingException;
@@ -30,17 +38,10 @@
import org.apache.distributedlog.io.Abortables;
import org.apache.distributedlog.io.AsyncAbortable;
import org.apache.distributedlog.io.AsyncCloseable;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.common.util.PermitManager;
import org.apache.distributedlog.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
abstract class BKAbstractLogWriter implements Closeable, AsyncCloseable, Abortable, AsyncAbortable {
static final Logger LOG = LoggerFactory.getLogger(BKAbstractLogWriter.class);
@@ -77,7 +78,7 @@
// manage write handler
- synchronized protected BKLogWriteHandler getCachedWriteHandler() {
+ protected synchronized BKLogWriteHandler getCachedWriteHandler() {
return writeHandler;
}
@@ -217,7 +218,7 @@
}
/**
- * Close the writer and release all the underlying resources
+ * Close the writer and release all the underlying resources.
*/
protected CompletableFuture<Void> closeNoThrow() {
CompletableFuture<Void> closeFuture;
@@ -280,7 +281,7 @@
}
// used by async writer
- synchronized protected CompletableFuture<BKLogSegmentWriter> asyncGetLedgerWriter(boolean resetOnError) {
+ protected synchronized CompletableFuture<BKLogSegmentWriter> asyncGetLedgerWriter(boolean resetOnError) {
final BKLogSegmentWriter ledgerWriter = getCachedLogWriter();
CompletableFuture<BKLogSegmentWriter> ledgerWriterFuture = getCachedLogWriterFuture();
if (null == ledgerWriterFuture || null == ledgerWriter) {
@@ -378,7 +379,8 @@
final long startTxId,
final boolean bestEffort,
final boolean allowMaxTxID) {
- final PermitManager.Permit switchPermit = bkDistributedLogManager.getLogSegmentRollingPermitManager().acquirePermit();
+ final PermitManager.Permit switchPermit =
+ bkDistributedLogManager.getLogSegmentRollingPermitManager().acquirePermit();
if (switchPermit.isAllowed()) {
return FutureUtils.ensure(
FutureUtils.rescue(
@@ -392,17 +394,22 @@
// rescue function
cause -> {
if (cause instanceof LockingException) {
- LOG.warn("We lost lock during completeAndClose log segment for {}. Disable ledger rolling until it is recovered : ",
+ LOG.warn("We lost lock during completeAndClose log segment for {}."
+ + "Disable ledger rolling until it is recovered : ",
writeHandler.getFullyQualifiedName(), cause);
- bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
+ bkDistributedLogManager.getLogSegmentRollingPermitManager()
+ .disallowObtainPermits(switchPermit);
return FutureUtils.value(oldSegmentWriter);
} else if (cause instanceof ZKException) {
ZKException zke = (ZKException) cause;
if (ZKException.isRetryableZKException(zke)) {
- LOG.warn("Encountered zookeeper connection issues during completeAndClose log segment for {}." +
- " Disable ledger rolling until it is recovered : {}", writeHandler.getFullyQualifiedName(),
+ LOG.warn("Encountered zookeeper connection issues during completeAndClose "
+ + "log segment for {}. "
+ + "Disable ledger rolling until it is recovered : {}",
+ writeHandler.getFullyQualifiedName(),
zke.getKeeperExceptionCode());
- bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
+ bkDistributedLogManager.getLogSegmentRollingPermitManager()
+ .disallowObtainPermits(switchPermit);
return FutureUtils.value(oldSegmentWriter);
}
}
@@ -441,7 +448,8 @@
return FutureUtils.value(oldSegmentWriter);
} else {
return FutureUtils.exception(
- new UnexpectedException("StartLogSegment returns null for bestEffort rolling"));
+ new UnexpectedException("StartLogSegment returns "
+ + "null for bestEffort rolling"));
}
}
cacheAllocatedLogWriter(newSegmentWriter);
@@ -480,7 +488,7 @@
return completePromise;
}
- synchronized protected CompletableFuture<BKLogSegmentWriter> rollLogSegmentIfNecessary(
+ protected synchronized CompletableFuture<BKLogSegmentWriter> rollLogSegmentIfNecessary(
final BKLogSegmentWriter segmentWriter,
long startTxId,
boolean bestEffort,
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
index d80555a..0c6735a 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
@@ -35,6 +35,8 @@
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.DLIllegalStateException;
import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.exceptions.EndOfStreamException;
@@ -42,8 +44,6 @@
import org.apache.distributedlog.exceptions.LogNotFoundException;
import org.apache.distributedlog.exceptions.ReadCancelledException;
import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Utils;
import org.slf4j.Logger;
@@ -78,11 +78,12 @@
protected final BKLogReadHandler readHandler;
private final AtomicReference<Throwable> lastException = new AtomicReference<Throwable>();
private final OrderedScheduler scheduler;
- private final ConcurrentLinkedQueue<PendingReadRequest> pendingRequests = new ConcurrentLinkedQueue<PendingReadRequest>();
+ private final ConcurrentLinkedQueue<PendingReadRequest> pendingRequests =
+ new ConcurrentLinkedQueue<PendingReadRequest>();
private final Object scheduleLock = new Object();
private final AtomicLong scheduleCount = new AtomicLong(0);
- final private Stopwatch scheduleDelayStopwatch;
- final private Stopwatch readNextDelayStopwatch;
+ private final Stopwatch scheduleDelayStopwatch;
+ private final Stopwatch readNextDelayStopwatch;
private DLSN startDLSN;
private ReadAheadEntryReader readAheadReader = null;
private int lastPosition = 0;
@@ -191,7 +192,8 @@
void complete() {
if (LOG.isTraceEnabled()) {
- LOG.trace("{} : Satisfied promise with {} records", readHandler.getFullyQualifiedName(), records.size());
+ LOG.trace("{} : Satisfied promise with {} records",
+ readHandler.getFullyQualifiedName(), records.size());
}
delayUntilPromiseSatisfied.registerSuccessfulEvent(
enqueueTime.stop().elapsed(TimeUnit.MICROSECONDS), TimeUnit.MICROSECONDS);
@@ -277,12 +279,12 @@
// - cache is empty and readahead is idle (no records added for a long time)
idleReaderCheckIdleReadAheadCount.inc();
try {
- if (null == readAheadReader || (!hasMoreRecords() &&
- readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS))) {
+ if (null == readAheadReader || (!hasMoreRecords()
+ && readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS))) {
markReaderAsIdle();
return;
} else if (lastProcessTime.elapsed(TimeUnit.MILLISECONDS) > idleErrorThresholdMillis) {
- markReaderAsIdle();;
+ markReaderAsIdle();
}
} catch (IOException e) {
setLastException(e);
@@ -305,7 +307,8 @@
idleReaderTimeoutTask.cancel(true);
}
} catch (Exception exc) {
- LOG.info("{}: Failed to cancel the background idle reader timeout task", readHandler.getFullyQualifiedName());
+ LOG.info("{}: Failed to cancel the background idle reader timeout task",
+ readHandler.getFullyQualifiedName());
}
}
@@ -313,7 +316,7 @@
idleReaderError.inc();
IdleReaderException ire = new IdleReaderException("Reader on stream "
+ readHandler.getFullyQualifiedName()
- + " is idle for " + idleErrorThresholdMillis +" ms");
+ + " is idle for " + idleErrorThresholdMillis + " ms");
setLastException(ire);
// cancel all pending reads directly rather than notifying on error
// because idle reader could happen on idle read requests that usually means something wrong
@@ -565,7 +568,7 @@
@Override
public void run() {
- synchronized(scheduleLock) {
+ synchronized (scheduleLock) {
if (scheduleDelayStopwatch.isRunning()) {
scheduleLatency.registerSuccessfulEvent(
scheduleDelayStopwatch.stop().elapsed(TimeUnit.MICROSECONDS), TimeUnit.MICROSECONDS);
@@ -575,13 +578,13 @@
int iterations = 0;
long scheduleCountLocal = scheduleCount.get();
LOG.debug("{}: Scheduled Background Reader", readHandler.getFullyQualifiedName());
- while(true) {
+ while (true) {
if (LOG.isTraceEnabled()) {
LOG.trace("{}: Executing Iteration: {}", readHandler.getFullyQualifiedName(), iterations++);
}
PendingReadRequest nextRequest = null;
- synchronized(this) {
+ synchronized (this) {
nextRequest = pendingRequests.peek();
// Queue is empty, nothing to read, return
@@ -630,7 +633,8 @@
// read single record
do {
record = readNextRecord();
- } while (null != record && (record.isControl() || (record.getDlsn().compareTo(getStartDLSN()) < 0)));
+ } while (null != record && (record.isControl()
+ || (record.getDlsn().compareTo(getStartDLSN()) < 0)));
if (null == record) {
break;
} else {
@@ -642,20 +646,22 @@
// gap detection
if (recordPositionsContainsGap(record, lastPosition)) {
- bkDistributedLogManager.raiseAlert("Gap detected between records at record = {}", record);
+ bkDistributedLogManager.raiseAlert("Gap detected between records at record = {}",
+ record);
if (positionGapDetectionEnabled) {
- throw new DLIllegalStateException("Gap detected between records at record = " + record);
+ throw new DLIllegalStateException("Gap detected between records at record = "
+ + record);
}
}
lastPosition = record.getLastPositionWithinLogSegment();
-
nextRequest.addRecord(record);
}
- };
+ }
} catch (IOException exc) {
setLastException(exc);
if (!(exc instanceof LogNotFoundException)) {
- LOG.warn("{} : read with skip Exception", readHandler.getFullyQualifiedName(), lastException.get());
+ LOG.warn("{} : read with skip Exception",
+ readHandler.getFullyQualifiedName(), lastException.get());
}
continue;
}
@@ -715,12 +721,12 @@
final boolean emptyLogSegment = (0 == lastPosition);
final boolean positionIncreasedByOne = (record.getPositionWithinLogSegment() == (lastPosition + 1));
- return !firstLogRecord && !endOfStreamRecord && !emptyLogSegment &&
- !positionIncreasedByOne;
+ return !firstLogRecord && !endOfStreamRecord && !emptyLogSegment
+ && !positionIncreasedByOne;
}
/**
- * Triggered when the background activity encounters an exception
+ * Triggered when the background activity encounters an exception.
*/
@Override
public void notifyOnError(Throwable cause) {
@@ -733,7 +739,7 @@
}
/**
- * Triggered when the background activity completes an operation
+ * Triggered when the background activity completes an operation.
*/
@Override
public void notifyOnOperationComplete() {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
index abcc4c4..a492f51 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
@@ -17,8 +17,8 @@
*/
package org.apache.distributedlog;
-import com.google.common.base.Stopwatch;
import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
@@ -32,14 +32,14 @@
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.exceptions.StreamNotReadyException;
import org.apache.distributedlog.exceptions.WriteCancelledException;
import org.apache.distributedlog.exceptions.WriteException;
import org.apache.distributedlog.feature.CoreFeatureKeys;
import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -67,7 +67,7 @@
static final Logger LOG = LoggerFactory.getLogger(BKAsyncLogWriter.class);
- static Function<List<LogSegmentMetadata>, Boolean> TruncationResultConverter =
+ static Function<List<LogSegmentMetadata>, Boolean> truncationResultConverter =
segments -> true;
// Records pending for roll log segment.
@@ -98,7 +98,6 @@
/**
* Last pending record in current log segment. After it is satisified, it would
* roll log segment.
- *
* This implementation is based on the assumption that all future satisified in same
* order future pool.
*/
@@ -151,7 +150,8 @@
this.disableRollOnSegmentError = conf.getDisableRollingOnLogSegmentError();
// features
- disableLogSegmentRollingFeature = featureProvider.getFeature(CoreFeatureKeys.DISABLE_LOGSEGMENT_ROLLING.name().toLowerCase());
+ disableLogSegmentRollingFeature = featureProvider
+ .getFeature(CoreFeatureKeys.DISABLE_LOGSEGMENT_ROLLING.name().toLowerCase());
// stats
this.statsLogger = dlmStatsLogger.scope("log_writer");
this.writeOpStatsLogger = statsLogger.getOpStatsLogger("write");
@@ -172,7 +172,8 @@
}
/**
- * Write a log record as control record. The method will be used by Monitor Service to enforce a new inprogress segment.
+ * Write a log record as control record.
+ * The method will be used by Monitor Service to enforce a new inprogress segment.
*
* @param record
* log record
@@ -253,9 +254,9 @@
boolean shouldRollLog(BKLogSegmentWriter w) {
try {
- return null == w ||
- (!disableLogSegmentRollingFeature.isAvailable() &&
- shouldStartNewSegment(w));
+ return null == w
+ || (!disableLogSegmentRollingFeature.isAvailable()
+ && shouldStartNewSegment(w));
} catch (IOException ioe) {
return false;
}
@@ -445,7 +446,7 @@
} catch (IOException e) {
return FutureUtils.exception(e);
}
- return writeHandler.setLogSegmentsOlderThanDLSNTruncated(dlsn).thenApply(TruncationResultConverter);
+ return writeHandler.setLogSegmentsOlderThanDLSNTruncated(dlsn).thenApply(truncationResultConverter);
}
CompletableFuture<Long> flushAndCommit() {
@@ -501,7 +502,7 @@
/**
* *TEMP HACK*
- * Get the name of the stream this writer writes data to
+ * Get the name of the stream this writer writes data to.
*/
@Override
public String getStreamName() {
@@ -514,7 +515,8 @@
synchronized (this) {
if (pendingRequests != null) {
for (PendingLogRecord pendingLogRecord : pendingRequests) {
- pendingLogRecord.promise.completeExceptionally(new WriteException(bkDistributedLogManager.getStreamName(),
+ pendingLogRecord.promise
+ .completeExceptionally(new WriteException(bkDistributedLogManager.getStreamName(),
"abort wring: writer has been closed due to error."));
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
index 0289624..d5d1477 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
@@ -17,21 +17,31 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.distributedlog.namespace.NamespaceDriver.Role.READER;
import static org.apache.distributedlog.namespace.NamespaceDriver.Role.WRITER;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
+
import java.io.IOException;
import java.net.URI;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
-import java.util.concurrent.ExecutorService;
+
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+
import java.util.function.Function;
+
+
+
+
+
+
+
import org.apache.bookkeeper.feature.FeatureProvider;
import org.apache.bookkeeper.stats.AlertStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
@@ -39,38 +49,48 @@
import org.apache.distributedlog.api.AsyncLogWriter;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
+
import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.stats.BroadCastStatsLogger;
+
+import org.apache.distributedlog.common.util.PermitLimiter;
+import org.apache.distributedlog.common.util.PermitManager;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+
import org.apache.distributedlog.exceptions.AlreadyClosedException;
import org.apache.distributedlog.exceptions.LogEmptyException;
import org.apache.distributedlog.exceptions.LogNotFoundException;
import org.apache.distributedlog.exceptions.UnexpectedException;
import org.apache.distributedlog.injector.AsyncFailureInjector;
-import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
-import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
-import org.apache.distributedlog.metadata.LogMetadataForReader;
-import org.apache.distributedlog.metadata.LogMetadataForWriter;
import org.apache.distributedlog.io.AsyncCloseable;
import org.apache.distributedlog.lock.DistributedLock;
import org.apache.distributedlog.lock.NopDistributedLock;
import org.apache.distributedlog.lock.ZKDistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
import org.apache.distributedlog.logsegment.LogSegmentFilter;
+
import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
+
+import org.apache.distributedlog.metadata.LogMetadataForReader;
+import org.apache.distributedlog.metadata.LogMetadataForWriter;
+
import org.apache.distributedlog.metadata.LogStreamMetadataStore;
import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.common.stats.BroadCastStatsLogger;
-import org.apache.distributedlog.api.subscription.SubscriptionsStore;
import org.apache.distributedlog.util.Allocator;
import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.common.util.PermitLimiter;
-import org.apache.distributedlog.common.util.PermitManager;
+
import org.apache.distributedlog.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
+
/**
* <h3>Metrics</h3>
* <ul>
@@ -422,7 +442,7 @@
}
/**
- * Check if an end of stream marker was added to the stream
+ * 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
@@ -435,7 +455,7 @@
}
/**
- * Begin appending to the end of the log stream which is being treated as a sequence of bytes
+ * Begin appending to the end of the log stream which is being treated as a sequence of bytes.
*
* @return the writer interface to generate log records
*/
@@ -443,8 +463,8 @@
long position;
try {
position = Utils.ioResult(getLastLogRecordAsyncInternal(true, false)).getTransactionId();
- if (DistributedLogConstants.INVALID_TXID == position ||
- DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID == position) {
+ if (DistributedLogConstants.INVALID_TXID == position
+ || DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID == position) {
position = 0;
}
} catch (LogEmptyException ex) {
@@ -456,7 +476,7 @@
}
/**
- * Get a reader to read a log stream as a sequence of bytes
+ * Get a reader to read a log stream as a sequence of bytes.
*
* @return the writer interface to generate log records
*/
@@ -465,7 +485,7 @@
}
/**
- * Begin writing to the log stream identified by the name
+ * Begin writing to the log stream identified by the name.
*
* @return the writer interface to generate log records
*/
@@ -493,7 +513,7 @@
}
/**
- * Begin writing to the log stream identified by the name
+ * Begin writing to the log stream identified by the name.
*
* @return the writer interface to generate log records
*/
@@ -599,7 +619,7 @@
}
/**
- * Get the input stream starting with fromTxnId for the specified log
+ * Get the input stream starting with fromTxnId for the specified log.
*
* @param fromTxnId - the first transaction id we want to read
* @return the stream starting with transaction fromTxnId
@@ -648,7 +668,8 @@
* </p>
*
* @see DLUtils#findLogSegmentNotLessThanTxnId(List, long)
- * @see ReadUtils#getLogRecordNotLessThanTxId(String, LogSegmentMetadata, long, ExecutorService, LogSegmentEntryStore, int)
+ * @see ReadUtils#getLogRecordNotLessThanTxId(String, LogSegmentMetadata,
+ * long, ExecutorService, LogSegmentEntryStore, int)
* @param fromTxnId
* transaction id to start reading from
* @return future representing the open result.
@@ -779,7 +800,7 @@
}
/**
- * Get the input stream starting with fromTxnId for the specified log
+ * Get the input stream starting with fromTxnId for the specified log.
*
* @param fromTxnId
* transaction id to start reading from
@@ -810,7 +831,7 @@
}
/**
- * Get the last log record in the stream
+ * 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.
@@ -840,7 +861,7 @@
}
/**
- * Get Latest log record in the log
+ * Get Latest log record in the log.
*
* @return latest log record
*/
@@ -860,7 +881,7 @@
}
/**
- * Get Latest Transaction Id in the log
+ * Get Latest Transaction Id in the log.
*
* @return latest transaction id
*/
@@ -901,7 +922,7 @@
}
/**
- * Get the number of log records in the active portion of the log
+ * 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
@@ -955,7 +976,7 @@
}
/**
- * Delete all the partitions of the specified log
+ * Delete all the partitions of the specified log.
*
* @throws IOException if the deletion fails
*/
@@ -981,7 +1002,7 @@
*/
@Override
public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException {
- Preconditions.checkArgument(minTxIdToKeep > 0, "Invalid transaction id " + minTxIdToKeep);
+ checkArgument(minTxIdToKeep > 0, "Invalid transaction id " + minTxIdToKeep);
checkClosedOrInError("purgeLogSegmentsOlderThan");
BKLogWriteHandler ledgerHandler = createWriteHandler(true);
try {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
index 0264178..bc6cc8b 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
@@ -27,10 +27,14 @@
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.distributedlog.acl.AccessControlManager;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.namespace.Namespace;
import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.common.util.PermitLimiter;
+import org.apache.distributedlog.common.util.SchedulerUtils;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.exceptions.AlreadyClosedException;
import org.apache.distributedlog.exceptions.InvalidStreamNameException;
@@ -40,11 +44,8 @@
import org.apache.distributedlog.namespace.NamespaceDriver;
import org.apache.distributedlog.util.ConfUtils;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.common.util.PermitLimiter;
-import org.apache.distributedlog.common.util.SchedulerUtils;
import org.apache.distributedlog.util.Utils;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -76,7 +77,6 @@
* </ul>
*
* <h4>DistributedLogManager</h4>
- *
* All the core stats about reader and writer are exposed under current scope via {@link BKDistributedLogManager}.
*/
public class BKDistributedLogNamespace implements Namespace {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
index 9f2e750..ddf0734 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
@@ -40,20 +40,23 @@
import org.apache.bookkeeper.versioning.Versioned;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.LogEmptyException;
import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
-import org.apache.distributedlog.metadata.LogMetadata;
import org.apache.distributedlog.io.AsyncAbortable;
import org.apache.distributedlog.io.AsyncCloseable;
-import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
-import org.apache.distributedlog.logsegment.PerStreamLogSegmentCache;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
import org.apache.distributedlog.logsegment.LogSegmentFilter;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
+import org.apache.distributedlog.logsegment.PerStreamLogSegmentCache;
+
+import org.apache.distributedlog.metadata.LogMetadata;
+
import org.apache.distributedlog.metadata.LogStreamMetadataStore;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+
import org.apache.distributedlog.util.OrderedScheduler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -65,8 +68,8 @@
* The log handler is a base class on managing log segments. so all the metrics
* here are related to log segments retrieval and exposed under `logsegments`.
* These metrics are all OpStats, in the format of <code>`scope`/logsegments/`op`</code>.
- * <p>
- * Those operations are:
+ *
+ * <p>Those operations are:
* <ul>
* <li>get_inprogress_segment: time between the inprogress log segment created and
* the handler read it.
@@ -186,7 +189,8 @@
@Override
public void onSuccess(Versioned<List<LogSegmentMetadata>> ledgerList) {
if (ledgerList.getValue().isEmpty()) {
- promise.completeExceptionally(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+ promise.completeExceptionally(new LogEmptyException("Log "
+ + getFullyQualifiedName() + " has no records"));
return;
}
CompletableFuture<LogRecordWithDLSN> firstRecord = null;
@@ -199,7 +203,8 @@
if (null != firstRecord) {
FutureUtils.proxyTo(firstRecord, promise);
} else {
- promise.completeExceptionally(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+ promise.completeExceptionally(new LogEmptyException("Log "
+ + getFullyQualifiedName() + " has no records"));
}
}
@@ -218,7 +223,8 @@
return promise;
}
- public CompletableFuture<LogRecordWithDLSN> getLastLogRecordAsync(final boolean recover, final boolean includeEndOfStream) {
+ public CompletableFuture<LogRecordWithDLSN> getLastLogRecordAsync(final boolean recover,
+ final boolean includeEndOfStream) {
final CompletableFuture<LogRecordWithDLSN> promise = new CompletableFuture<LogRecordWithDLSN>();
streamMetadataStore.logExists(
logMetadata.getUri(),
@@ -553,7 +559,7 @@
}
/**
- * Update the log segment cache with updated mapping
+ * Update the log segment cache with updated mapping.
*
* @param logSegmentsRemoved log segments removed
* @param logSegmentsAdded log segments added
@@ -570,7 +576,7 @@
}
/**
- * Read the log segments from the store and register a listener
+ * Read the log segments from the store and register a listener.
* @param comparator
* @param segmentFilter
* @param logSegmentNamesListener
@@ -692,7 +698,8 @@
private void completeReadLogSegmentsFromStore(final Set<String> removedSegments,
final Map<String, LogSegmentMetadata> addedSegments,
final Comparator<LogSegmentMetadata> comparator,
- final CompletableFuture<Versioned<List<LogSegmentMetadata>>> readResult,
+ final CompletableFuture<Versioned<List<LogSegmentMetadata>>>
+ readResult,
final Version logSegmentNamesVersion,
final AtomicInteger numChildren,
final AtomicInteger numFailures) {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
index ff6b527..5ebdd3b 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
@@ -35,20 +35,22 @@
import org.apache.bookkeeper.versioning.Versioned;
import org.apache.distributedlog.callback.LogSegmentListener;
import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.exceptions.DLIllegalStateException;
import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.exceptions.LogNotFoundException;
import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
-import org.apache.distributedlog.metadata.LogMetadataForReader;
import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
import org.apache.distributedlog.logsegment.LogSegmentFilter;
import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
+import org.apache.distributedlog.metadata.LogMetadataForReader;
import org.apache.distributedlog.metadata.LogStreamMetadataStore;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+
+
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Utils;
import org.slf4j.Logger;
@@ -218,8 +220,8 @@
*/
void checkReadLock() throws DLIllegalStateException, LockingException {
synchronized (this) {
- if ((null == lockAcquireFuture) ||
- (!lockAcquireFuture.isDone())) {
+ if ((null == lockAcquireFuture)
+ || (!lockAcquireFuture.isDone())) {
throw new DLIllegalStateException("Attempt to check for lock before it has been acquired successfully");
}
}
@@ -268,9 +270,9 @@
this).whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
@Override
public void onFailure(Throwable cause) {
- if (cause instanceof LogNotFoundException ||
- cause instanceof LogSegmentNotFoundException ||
- cause instanceof UnexpectedException) {
+ if (cause instanceof LogNotFoundException
+ || cause instanceof LogSegmentNotFoundException
+ || cause instanceof UnexpectedException) {
// indicate some inconsistent behavior, abort
metadataException.compareAndSet(null, (IOException) cause);
// notify the reader that read handler is in error state
@@ -302,8 +304,8 @@
@Override
public void onSegmentsUpdated(final Versioned<List<String>> segments) {
synchronized (this) {
- if (lastNotifiedLogSegments.getVersion() != Version.NEW &&
- lastNotifiedLogSegments.getVersion().compare(segments.getVersion()) != Version.Occurred.BEFORE) {
+ if (lastNotifiedLogSegments.getVersion() != Version.NEW
+ && lastNotifiedLogSegments.getVersion().compare(segments.getVersion()) != Version.Occurred.BEFORE) {
// the log segments has been read, and it is possibly a retry from last segments update
return;
}
@@ -314,9 +316,9 @@
readLogSegmentsPromise.whenComplete(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
@Override
public void onFailure(Throwable cause) {
- if (cause instanceof LogNotFoundException ||
- cause instanceof LogSegmentNotFoundException ||
- cause instanceof UnexpectedException) {
+ if (cause instanceof LogNotFoundException
+ || cause instanceof LogSegmentNotFoundException
+ || cause instanceof UnexpectedException) {
// indicate some inconsistent behavior, abort
metadataException.compareAndSet(null, (IOException) cause);
// notify the reader that read handler is in error state
@@ -336,8 +338,9 @@
List<LogSegmentMetadata> segmentsToNotify = null;
synchronized (BKLogReadHandler.this) {
Versioned<List<LogSegmentMetadata>> lastLogSegments = lastNotifiedLogSegments;
- if (lastLogSegments.getVersion() == Version.NEW ||
- lastLogSegments.getVersion().compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
+ if (lastLogSegments.getVersion() == Version.NEW
+ || lastLogSegments.getVersion().compare(logSegments.getVersion())
+ == Version.Occurred.BEFORE) {
lastNotifiedLogSegments = logSegments;
segmentsToNotify = logSegments.getValue();
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
index e2178dc..1d65d07 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
@@ -18,8 +18,8 @@
package org.apache.distributedlog;
import static com.google.common.base.Charsets.UTF_8;
-import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Stopwatch;
@@ -35,32 +35,6 @@
import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Function;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.BKTransmitException;
-import org.apache.distributedlog.exceptions.EndOfStreamException;
-import org.apache.distributedlog.exceptions.FlushException;
-import org.apache.distributedlog.exceptions.LockingException;
-import org.apache.distributedlog.exceptions.LogRecordTooLongException;
-import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
-import org.apache.distributedlog.exceptions.WriteCancelledException;
-import org.apache.distributedlog.exceptions.WriteException;
-import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
-import org.apache.distributedlog.feature.CoreFeatureKeys;
-import org.apache.distributedlog.injector.FailureInjector;
-import org.apache.distributedlog.injector.RandomDelayFailureInjector;
-import org.apache.distributedlog.io.CompressionCodec;
-import org.apache.distributedlog.io.CompressionUtils;
-import org.apache.distributedlog.lock.DistributedLock;
-import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
-import org.apache.distributedlog.logsegment.LogSegmentWriter;
-import org.apache.distributedlog.common.stats.OpStatsListener;
-import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.common.util.PermitLimiter;
-import org.apache.distributedlog.util.SimplePermitLimiter;
-import org.apache.distributedlog.common.util.Sizable;
import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
import org.apache.bookkeeper.client.BKException;
@@ -73,13 +47,41 @@
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.util.MathUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.stats.OpStatsListener;
+import org.apache.distributedlog.common.util.PermitLimiter;
+import org.apache.distributedlog.common.util.Sizable;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.FlushException;
+import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
+import org.apache.distributedlog.exceptions.WriteCancelledException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.feature.CoreFeatureKeys;
+import org.apache.distributedlog.injector.FailureInjector;
+import org.apache.distributedlog.injector.RandomDelayFailureInjector;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.io.CompressionUtils;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
+import org.apache.distributedlog.logsegment.LogSegmentWriter;
+import org.apache.distributedlog.util.FailpointUtils;
+
+
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.SimplePermitLimiter;
+
import org.apache.distributedlog.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* BookKeeper Based Log Segment Writer.
- *
* Multiple log records are packed into a single bookkeeper
* entry before sending it over the network. The fact that the log record entries
* are complete in the bookkeeper entries means that each bookkeeper log entry
@@ -112,8 +114,8 @@
protected final LogSegmentEntryWriter entryWriter;
private final CompressionCodec.Type compressionType;
private final ReentrantLock transmitLock = new ReentrantLock();
- private final AtomicInteger transmitResult
- = new AtomicInteger(BKException.Code.OK);
+ private final AtomicInteger transmitResult =
+ new AtomicInteger(BKException.Code.OK);
private final DistributedLock lock;
private final boolean isDurableWriteEnabled;
private DLSN lastDLSN = DLSN.InvalidDLSN;
@@ -130,16 +132,18 @@
// Indicates whether there are writes that have been successfully transmitted that would need
// a control record to be transmitted to make them visible to the readers by updating the last
// add confirmed
- volatile private boolean controlFlushNeeded = false;
+ private volatile boolean controlFlushNeeded = false;
private boolean immediateFlushEnabled = false;
private int minDelayBetweenImmediateFlushMs = 0;
private Stopwatch lastTransmit;
private boolean streamEnded = false;
private final ScheduledFuture<?> periodicFlushSchedule;
private final ScheduledFuture<?> periodicKeepAliveSchedule;
- final private AtomicReference<ScheduledFuture<?>> transmitSchedFutureRef = new AtomicReference<ScheduledFuture<?>>(null);
- final private AtomicReference<ScheduledFuture<?>> immFlushSchedFutureRef = new AtomicReference<ScheduledFuture<?>>(null);
- final private AtomicReference<Exception> scheduledFlushException = new AtomicReference<Exception>(null);
+ private final AtomicReference<ScheduledFuture<?>> transmitSchedFutureRef =
+ new AtomicReference<ScheduledFuture<?>>(null);
+ private final AtomicReference<ScheduledFuture<?>> immFlushSchedFutureRef =
+ new AtomicReference<ScheduledFuture<?>>(null);
+ private final AtomicReference<Exception> scheduledFlushException = new AtomicReference<Exception>(null);
private boolean enforceLock = true;
private CompletableFuture<Void> closeFuture = null;
private final boolean enableRecordCounts;
@@ -299,7 +303,7 @@
int periodicFlushFrequency = configuredPeriodicFlushFrequency;
if (periodicFlushFrequency > 0 && scheduler != null) {
periodicFlushSchedule = scheduler.scheduleAtFixedRate(this,
- periodicFlushFrequency/2, periodicFlushFrequency/2, TimeUnit.MILLISECONDS);
+ periodicFlushFrequency / 2, periodicFlushFrequency / 2, TimeUnit.MILLISECONDS);
} else {
periodicFlushSchedule = null;
}
@@ -418,7 +422,7 @@
private synchronized int getAverageTransmitSize() {
if (numFlushesSinceRestart > 0) {
- long ret = numBytes/numFlushesSinceRestart;
+ long ret = numBytes / numFlushesSinceRestart;
if (ret < Integer.MIN_VALUE || ret > Integer.MAX_VALUE) {
throw new IllegalArgumentException
@@ -539,8 +543,8 @@
private void abortTransmitPacketOnClose(final boolean abort,
final AtomicReference<Throwable> throwExc,
final CompletableFuture<Void> closePromise) {
- LOG.info("Closing BKPerStreamLogWriter (abort={}) for {} :" +
- " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {}",
+ LOG.info("Closing BKPerStreamLogWriter (abort={}) for {} :"
+ + " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {}",
new Object[]{abort, fullyQualifiedLogSegment, getLastDLSN(),
outstandingTransmits.get(), getWritesPendingTransmit()});
@@ -587,8 +591,8 @@
public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
if (BKException.Code.OK != rc && BKException.Code.LedgerClosedException != rc) {
if (!abort) {
- throwExc.set(new IOException("Failed to close ledger for " + fullyQualifiedLogSegment + " : " +
- BKException.getMessage(rc)));
+ throwExc.set(new IOException("Failed to close ledger for "
+ + fullyQualifiedLogSegment + " : " + BKException.getMessage(rc)));
}
}
completeClosePromise(abort, throwExc, closePromise);
@@ -615,17 +619,17 @@
}
@Override
- synchronized public void write(LogRecord record) throws IOException {
+ public synchronized void write(LogRecord record) throws IOException {
writeUserRecord(record);
flushIfNeeded();
}
@Override
- synchronized public CompletableFuture<DLSN> asyncWrite(LogRecord record) {
+ public synchronized CompletableFuture<DLSN> asyncWrite(LogRecord record) {
return asyncWrite(record, true);
}
- synchronized public CompletableFuture<DLSN> asyncWrite(LogRecord record, boolean flush) {
+ public synchronized CompletableFuture<DLSN> asyncWrite(LogRecord record, boolean flush) {
CompletableFuture<DLSN> result = null;
try {
if (record.isControl()) {
@@ -662,9 +666,10 @@
return result;
}
- synchronized private CompletableFuture<DLSN> writeUserRecord(LogRecord record) throws IOException {
+ private synchronized CompletableFuture<DLSN> writeUserRecord(LogRecord record) throws IOException {
if (null != closeFuture) {
- throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(BKException.Code.LedgerClosedException));
+ throw new WriteException(fullyQualifiedLogSegment,
+ BKException.getMessage(BKException.Code.LedgerClosedException));
}
if (BKException.Code.OK != transmitResult.get()) {
@@ -676,8 +681,8 @@
throw new EndOfStreamException("Writing to a stream after it has been marked as completed");
}
- if ((record.getTransactionId() < 0) ||
- (record.getTransactionId() == DistributedLogConstants.MAX_TXID)) {
+ if ((record.getTransactionId() < 0)
+ || (record.getTransactionId() == DistributedLogConstants.MAX_TXID)) {
throw new TransactionIdOutOfOrderException(record.getTransactionId());
}
@@ -727,11 +732,11 @@
}
boolean shouldFailCompleteLogSegment() {
- return (transmitResult.get() != BKException.Code.OK) &&
- (transmitResult.get() != BKException.Code.LedgerClosedException);
+ return (transmitResult.get() != BKException.Code.OK)
+ && (transmitResult.get() != BKException.Code.LedgerClosedException);
}
- synchronized public CompletableFuture<DLSN> writeInternal(LogRecord record)
+ public synchronized CompletableFuture<DLSN> writeInternal(LogRecord record)
throws LogRecordTooLongException, LockingException, BKTransmitException,
WriteException, InvalidEnvelopedEntryException {
int logRecordSize = record.getPersistentSize();
@@ -772,7 +777,7 @@
return writePromise;
}
- synchronized private CompletableFuture<DLSN> writeControlLogRecord()
+ private synchronized CompletableFuture<DLSN> writeControlLogRecord()
throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
LockingException, LogRecordTooLongException {
LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.CONTROL_RECORD_CONTENT);
@@ -780,7 +785,7 @@
return writeControlLogRecord(controlRec);
}
- synchronized private CompletableFuture<DLSN> writeControlLogRecord(LogRecord record)
+ private synchronized CompletableFuture<DLSN> writeControlLogRecord(LogRecord record)
throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
LockingException, LogRecordTooLongException {
return writeInternal(record);
@@ -793,7 +798,7 @@
*
* @throws IOException
*/
- synchronized private void writeEndOfStreamMarker() throws IOException {
+ private synchronized void writeEndOfStreamMarker() throws IOException {
LogRecord endOfStreamRec = new LogRecord(DistributedLogConstants.MAX_TXID, "endOfStream".getBytes(UTF_8));
endOfStreamRec.setEndOfStream();
writeInternal(endOfStreamRec);
@@ -819,14 +824,13 @@
/**
* Write bulk of records.
- *
* (TODO: moved this method to log writer level)
*
* @param records list of records to write
* @return number of records that has been written
* @throws IOException when there is I/O errors during writing records.
*/
- synchronized public int writeBulk(List<LogRecord> records) throws IOException {
+ public synchronized int writeBulk(List<LogRecord> records) throws IOException {
int numRecords = 0;
for (LogRecord r : records) {
write(r);
@@ -932,7 +936,7 @@
scheduledFutureRef.set(scheduler.schedule(new Runnable() {
@Override
public void run() {
- synchronized(this) {
+ synchronized (this) {
scheduledFutureRef.set(null);
try {
callable.call();
@@ -997,7 +1001,6 @@
* Transmit the current buffer to bookkeeper.
* Synchronised at the class. #write() and #flush()
* are never called at the same time.
- *
* NOTE: This method should only throw known exceptions so that we don't accidentally
* add new code that throws in an inappropriate place.
*
@@ -1023,8 +1026,8 @@
fullyQualifiedLogSegment,
BKException.getMessage(transmitResult.get()));
throw new BKTransmitException("Trying to write to an errored stream;"
- + " Error code : (" + transmitResult.get()
- + ") " + BKException.getMessage(transmitResult.get()), transmitResult.get());
+ + " Error code : (" + transmitResult.get() + ") "
+ + BKException.getMessage(transmitResult.get()), transmitResult.get());
}
if (recordSetWriter.getNumRecords() == 0) {
@@ -1089,10 +1092,10 @@
}
/**
- * Checks if there is any data to transmit so that the periodic flush
- * task can determine if there is anything it needs to do
+ * Checks if there is any data to transmit so that the periodic
+ * flush task can determine if there is anything it needs to do.
*/
- synchronized private boolean haveDataToTransmit() {
+ private synchronized boolean haveDataToTransmit() {
if (!transmitResult.compareAndSet(BKException.Code.OK, BKException.Code.OK)) {
// Even if there is data it cannot be transmitted, so effectively nothing to send
return false;
@@ -1165,7 +1168,9 @@
@Override
public void onFailure(Throwable cause) {
LOG.error("addComplete processing failed for {} entry {} lastTxId {} rc {} with error",
- new Object[] {fullyQualifiedLogSegment, entryId, transmitPacket.getRecordSet().getMaxTxId(), rc, cause});
+ new Object[] {
+ fullyQualifiedLogSegment, entryId,
+ transmitPacket.getRecordSet().getMaxTxId(), rc, cause});
}
});
// Race condition if we notify before the addComplete is enqueued.
@@ -1259,11 +1264,11 @@
}
@Override
- synchronized public void run() {
+ public synchronized void run() {
backgroundFlush(false);
}
- synchronized private void backgroundFlush(boolean controlFlushOnly) {
+ private synchronized void backgroundFlush(boolean controlFlushOnly) {
if (null != closeFuture) {
// if the log segment is closing, skip any background flushing
LOG.debug("Skip background flushing since log segment {} is closing.", getFullyQualifiedLogSegment());
@@ -1290,7 +1295,7 @@
}
}
- synchronized private void keepAlive() {
+ private synchronized void keepAlive() {
if (null != closeFuture) {
// if the log segment is closing, skip sending any keep alive records.
LOG.debug("Skip sending keepAlive control record since log segment {} is closing.",
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
index 4c0019e..18ec549 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
@@ -36,6 +36,9 @@
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.versioning.Version;
import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.PermitLimiter;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.exceptions.DLIllegalStateException;
import org.apache.distributedlog.exceptions.EndOfStreamException;
@@ -44,26 +47,25 @@
import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
import org.apache.distributedlog.exceptions.UnexpectedException;
import org.apache.distributedlog.function.GetLastTxIdFunction;
+import org.apache.distributedlog.lock.DistributedLock;
import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
-import org.apache.distributedlog.metadata.LogMetadataForWriter;
-import org.apache.distributedlog.lock.DistributedLock;
import org.apache.distributedlog.logsegment.LogSegmentFilter;
import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
import org.apache.distributedlog.logsegment.RollingPolicy;
import org.apache.distributedlog.logsegment.SizeBasedRollingPolicy;
import org.apache.distributedlog.logsegment.TimeBasedRollingPolicy;
+import org.apache.distributedlog.metadata.LogMetadataForWriter;
+import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
import org.apache.distributedlog.metadata.LogStreamMetadataStore;
import org.apache.distributedlog.metadata.MetadataUpdater;
-import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
import org.apache.distributedlog.util.Allocator;
import org.apache.distributedlog.util.DLUtils;
import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+
+
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Transaction;
-import org.apache.distributedlog.common.util.PermitLimiter;
import org.apache.distributedlog.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -362,7 +364,7 @@
}
/**
- * Delete the whole log and all log segments under the log
+ * Delete the whole log and all log segments under the log.
*/
void deleteLog() throws IOException {
lock.checkOwnershipAndReacquire();
@@ -374,7 +376,6 @@
* The caller could call this before any actions, which to hold the lock for
* the write handler of its whole lifecycle. The lock will only be released
* when closing the write handler.
- *
* This method is useful to prevent releasing underlying zookeeper lock during
* recovering/completing log segments. Releasing underlying zookeeper lock means
* 1) increase latency when re-lock on starting new log segment. 2) increase the
@@ -474,8 +475,9 @@
// We only skip log segment sequence number validation only when no log segments found &
// the maximum log segment sequence number is "UNASSIGNED".
- if (!logSegmentsFound &&
- (DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO == maxLogSegmentSequenceNo.getSequenceNumber())) {
+ if (!logSegmentsFound
+ && (DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO
+ == maxLogSegmentSequenceNo.getSequenceNumber())) {
// no ledger seqno stored in /ledgers before
LOG.info("No max ledger sequence number found while creating log segment {} for {}.",
logSegmentSeqNo, getFullyQualifiedName());
@@ -492,7 +494,8 @@
return logSegmentSeqNo;
}
- protected BKLogSegmentWriter doStartLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID) throws IOException {
+ protected BKLogSegmentWriter doStartLogSegment(long txId,
+ boolean bestEffort, boolean allowMaxTxID) throws IOException {
return Utils.ioResult(asyncStartLogSegment(txId, bestEffort, allowMaxTxID));
}
@@ -525,8 +528,8 @@
final boolean allowMaxTxID,
final CompletableFuture<BKLogSegmentWriter> promise) {
// validate the tx id
- if ((txId < 0) ||
- (!allowMaxTxID && (txId == DistributedLogConstants.MAX_TXID))) {
+ if ((txId < 0)
+ || (!allowMaxTxID && (txId == DistributedLogConstants.MAX_TXID))) {
FutureUtils.completeExceptionally(promise, new IOException("Invalid Transaction Id " + txId));
return;
}
@@ -535,11 +538,13 @@
if (txId < highestTxIdWritten) {
if (highestTxIdWritten == DistributedLogConstants.MAX_TXID) {
LOG.error("We've already marked the stream as ended and attempting to start a new log segment");
- FutureUtils.completeExceptionally(promise, new EndOfStreamException("Writing to a stream after it has been marked as completed"));
+ FutureUtils.completeExceptionally(promise,
+ new EndOfStreamException("Writing to a stream after it has been marked as completed"));
return;
} else {
LOG.error("We've already seen TxId {} the max TXId is {}", txId, highestTxIdWritten);
- FutureUtils.completeExceptionally(promise, new TransactionIdOutOfOrderException(txId, highestTxIdWritten));
+ FutureUtils.completeExceptionally(promise,
+ new TransactionIdOutOfOrderException(txId, highestTxIdWritten));
return;
}
}
@@ -694,14 +699,17 @@
@Override
public void onSuccess(Void value) {
// in theory closeToFinalize should throw exception if a stream is in error.
- // just in case, add another checking here to make sure we don't close log segment is a stream is in error.
+ // just in case, add another checking here to make sure
+ // we don't close log segment is a stream is in error.
if (writer.shouldFailCompleteLogSegment()) {
FutureUtils.completeExceptionally(promise,
- new IOException("LogSegmentWriter for " + writer.getFullyQualifiedLogSegment() + " is already in error."));
+ new IOException("LogSegmentWriter for " + writer.getFullyQualifiedLogSegment()
+ + " is already in error."));
return;
}
doCompleteAndCloseLogSegment(
- inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
+ inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(),
+ writer.getLogSegmentSequenceNumber()),
writer.getLogSegmentSequenceNumber(),
writer.getLogSegmentId(),
writer.getStartTxId(),
@@ -726,8 +734,8 @@
long lastTxId,
int recordCount)
throws IOException {
- return completeAndCloseLogSegment(inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo), logSegmentSeqNo,
- logSegmentId, firstTxId, lastTxId, recordCount, -1, -1);
+ return completeAndCloseLogSegment(inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo),
+ logSegmentSeqNo, logSegmentId, firstTxId, lastTxId, recordCount, -1, -1);
}
/**
@@ -784,7 +792,7 @@
}
/**
- * Close log segment
+ * Close log segment.
*
* @param inprogressZnodeName
* @param logSegmentSeqNo
@@ -894,12 +902,14 @@
return;
}
long leastInprogressLSSN = inprogressLSSNs.getFirst();
- // the log segment sequence number in metadata {@link inprogressLogSegment.getLogSegmentSequenceNumber()}
+ // the log segment sequence number in metadata
+ // {@link inprogressLogSegment.getLogSegmentSequenceNumber()}
// should be same as the sequence number we are completing (logSegmentSeqNo)
// and
- // it should also be same as the least inprogress log segment sequence number tracked in {@link inprogressLSSNs}
- if ((inprogressLogSegment.getLogSegmentSequenceNumber() != logSegmentSeqNo) ||
- (leastInprogressLSSN != logSegmentSeqNo)) {
+ // it should also be same as the least inprogress log segment sequence number
+ // tracked in {@link inprogressLSSNs}
+ if ((inprogressLogSegment.getLogSegmentSequenceNumber() != logSegmentSeqNo)
+ || (leastInprogressLSSN != logSegmentSeqNo)) {
FutureUtils.completeExceptionally(promise, new UnexpectedException(
"Didn't find matched inprogress log segments when completing inprogress "
+ inprogressLogSegment));
@@ -909,18 +919,20 @@
}
// store max sequence number.
- long maxSeqNo= Math.max(logSegmentSeqNo, maxLogSegmentSequenceNo.getSequenceNumber());
- if (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo ||
- (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo + 1)) {
+ long maxSeqNo = Math.max(logSegmentSeqNo, maxLogSegmentSequenceNo.getSequenceNumber());
+ if (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo
+ || (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo + 1)) {
// ignore the case that a new inprogress log segment is pre-allocated
// before completing current inprogress one
LOG.info("Try storing max sequence number {} in completing {}.",
new Object[] { logSegmentSeqNo, inprogressLogSegment.getZkPath() });
} else {
LOG.warn("Unexpected max ledger sequence number {} found while completing log segment {} for {}",
- new Object[] { maxLogSegmentSequenceNo.getSequenceNumber(), logSegmentSeqNo, getFullyQualifiedName() });
+ new Object[] {
+ maxLogSegmentSequenceNo.getSequenceNumber(), logSegmentSeqNo, getFullyQualifiedName() });
if (validateLogSegmentSequenceNumber) {
- FutureUtils.completeExceptionally(promise, new DLIllegalStateException("Unexpected max log segment sequence number "
+ FutureUtils.completeExceptionally(promise,
+ new DLIllegalStateException("Unexpected max log segment sequence number "
+ maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
+ ", expected " + (logSegmentSeqNo - 1)));
return;
@@ -982,7 +994,8 @@
} catch (IOException ioe) {
return FutureUtils.exception(ioe);
}
- return getCachedLogSegmentsAfterFirstFetch(LogSegmentMetadata.COMPARATOR).thenCompose(recoverLogSegmentsFunction);
+ return getCachedLogSegmentsAfterFirstFetch(LogSegmentMetadata.COMPARATOR)
+ .thenCompose(recoverLogSegmentsFunction);
}
class RecoverLogSegmentFunction implements Function<LogSegmentMetadata, CompletableFuture<LogSegmentMetadata>> {
@@ -1050,8 +1063,8 @@
logSegments -> setLogSegmentsOlderThanDLSNTruncated(logSegments, dlsn));
}
- private CompletableFuture<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(List<LogSegmentMetadata> logSegments,
- final DLSN dlsn) {
+ private CompletableFuture<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());
@@ -1066,11 +1079,13 @@
} else if (l.getFirstDLSN().compareTo(dlsn) < 0) {
// Can be satisfied by at most one segment
if (null != partialTruncate) {
- String logMsg = String.format("Potential metadata inconsistency for stream %s at segment %s", getFullyQualifiedName(), l);
+ String logMsg = String.format("Potential metadata inconsistency for stream %s at segment %s",
+ getFullyQualifiedName(), l);
LOG.error(logMsg);
return FutureUtils.exception(new DLIllegalStateException(logMsg));
}
- LOG.info("{}: Partially truncating log segment {} older than {}.", new Object[] {getFullyQualifiedName(), l, dlsn});
+ LOG.info("{}: Partially truncating log segment {} older than {}.",
+ new Object[] {getFullyQualifiedName(), l, dlsn});
partialTruncate = l;
} else {
break;
@@ -1117,8 +1132,8 @@
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)) {
+ 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
@@ -1147,9 +1162,9 @@
List<LogSegmentMetadata> purgeList = Lists.newArrayListWithExpectedSize(numLogSegmentsToProcess);
for (int iterator = 0; iterator < numLogSegmentsToProcess; iterator++) {
LogSegmentMetadata l = logSegments.get(iterator);
- if ((minTxIdToKeep < 0) ||
- ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) &&
- !l.isInProgress() && (l.getLastTxId() < minTxIdToKeep))) {
+ if ((minTxIdToKeep < 0)
+ || ((l.isTruncated() || !conf.getExplicitTruncationByApplication())
+ && !l.isInProgress() && (l.getLastTxId() < minTxIdToKeep))) {
purgeList.add(l);
} else {
// stop truncating log segments if we find either an inprogress or a partially
@@ -1168,7 +1183,7 @@
final List<LogSegmentMetadata> listToTruncate = Lists.newArrayListWithCapacity(truncateList.size() + 1);
final List<LogSegmentMetadata> listAfterTruncated = Lists.newArrayListWithCapacity(truncateList.size() + 1);
Transaction<Object> updateTxn = metadataUpdater.transaction();
- for(LogSegmentMetadata l : truncateList) {
+ for (LogSegmentMetadata l : truncateList) {
if (!l.isTruncated()) {
LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentTruncated(updateTxn, l);
listToTruncate.add(l);
@@ -1176,8 +1191,9 @@
}
}
- if (null != partialTruncate && (partialTruncate.isNonTruncated() ||
- (partialTruncate.isPartiallyTruncated() && (partialTruncate.getMinActiveDLSN().compareTo(minActiveDLSN) < 0)))) {
+ if (null != partialTruncate && (partialTruncate.isNonTruncated()
+ || (partialTruncate.isPartiallyTruncated()
+ && (partialTruncate.getMinActiveDLSN().compareTo(minActiveDLSN) < 0)))) {
LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentPartiallyTruncated(
updateTxn, partialTruncate, minActiveDLSN);
listToTruncate.add(partialTruncate);
@@ -1288,7 +1304,7 @@
}
/**
- * Get the znode path for a finalize ledger
+ * Get the znode path for a finalize ledger.
*/
String completedLedgerZNode(long firstTxId, long lastTxId, long logSegmentSeqNo) {
return String.format("%s/%s", logMetadata.getLogSegmentsPath(),
@@ -1310,7 +1326,7 @@
}
/**
- * Get the znode path for the inprogressZNode
+ * Get the znode path for the inprogressZNode.
*/
String inprogressZNode(long logSegmentId, long firstTxId, long logSegmentSeqNo) {
return logMetadata.getLogSegmentsPath() + "/" + inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
index fa56cfb..35457b2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
@@ -20,24 +20,26 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Ticker;
-import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.api.AsyncLogReader;
-import org.apache.distributedlog.api.LogReader;
-import org.apache.distributedlog.exceptions.EndOfStreamException;
-import org.apache.distributedlog.exceptions.IdleReaderException;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.Utils;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.IdleReaderException;
+import org.apache.distributedlog.util.Utils;
+
+
+
/**
- * Synchronous Log Reader based on {@link AsyncLogReader}
+ * Synchronous Log Reader based on {@link AsyncLogReader}.
*/
class BKSyncLogReader implements LogReader, AsyncNotification {
@@ -171,8 +173,8 @@
LogRecordWithDLSN record = doReadNext(nonBlocking);
// no record is returned, check if the reader becomes idle
if (null == record && shouldCheckIdleReader) {
- if (readAheadReader.getNumCachedEntries() <= 0 &&
- readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS)) {
+ if (readAheadReader.getNumCachedEntries() <= 0
+ && readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS)) {
markReaderAsIdle();
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
index 3715327..ef75a6d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
@@ -44,6 +44,7 @@
/**
* Complete the transmit with result code <code>transmitRc</code>.
+ *
* <p>It would notify all the waiters that are waiting via {@link #awaitTransmitComplete(long, TimeUnit)}
* or {@link #addTransmitCompleteListener(FutureEventListener)}.
*
@@ -56,6 +57,7 @@
/**
* Register a transmit complete listener.
+ *
* <p>The listener will be triggered with transmit result when transmit completes.
* The method should be non-blocking.
*
@@ -68,7 +70,7 @@
}
/**
- * Await for the transmit to be complete
+ * Await for the transmit to be complete.
*
* @param timeout
* wait timeout
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
index a50c391..d1c1d6f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
@@ -17,9 +17,11 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Charsets.UTF_8;
import com.google.common.base.Optional;
import io.netty.channel.EventLoopGroup;
import io.netty.util.HashedWheelTimer;
+import java.io.IOException;
import java.util.Collections;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.client.AsyncCallback;
@@ -36,19 +38,17 @@
import org.apache.commons.configuration.ConfigurationException;
import org.apache.distributedlog.ZooKeeperClient.Credentials;
import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.AlreadyClosedException;
import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.exceptions.ZKException;
import org.apache.distributedlog.net.NetUtils;
import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import static com.google.common.base.Charsets.UTF_8;
/**
* BookKeeper Client wrapper over {@link BookKeeper}.
@@ -159,8 +159,8 @@
}
this.zkc = new ZooKeeperClient(name + ":zk", zkSessionTimeout, 2 * zkSessionTimeout, zkServers,
- retryPolicy, statsLogger.scope("bkc_zkc"), conf.getZKClientNumberRetryThreads(),
- conf.getBKClientZKRequestRateLimit(), credentials);
+ retryPolicy, statsLogger.scope("bkc_zkc"),
+ conf.getZKClientNumberRetryThreads(), conf.getBKClientZKRequestRateLimit(), credentials);
}
try {
@@ -172,15 +172,15 @@
}
if (ownZK) {
- LOG.info("BookKeeper Client created {} with its own ZK Client : ledgersPath = {}, numRetries = {}, " +
- "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
+ LOG.info("BookKeeper Client created {} with its own ZK Client : ledgersPath = {}, numRetries = {}, "
+ + "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
new Object[] { name, ledgersPath,
conf.getBKClientZKNumRetries(), conf.getBKClientZKSessionTimeoutMilliSeconds(),
conf.getBKClientZKRetryBackoffStartMillis(), conf.getBKClientZKRetryBackoffMaxMillis(),
conf.getBkDNSResolverOverrides() });
} else {
- LOG.info("BookKeeper Client created {} with shared zookeeper client : ledgersPath = {}, numRetries = {}, " +
- "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
+ LOG.info("BookKeeper Client created {} with shared zookeeper client : ledgersPath = {}, numRetries = {}, "
+ + "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
new Object[] { name, ledgersPath,
conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(),
conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(),
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
index 1149ad5..d348287 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
@@ -17,13 +17,14 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
import io.netty.channel.EventLoopGroup;
import io.netty.util.HashedWheelTimer;
+import org.apache.bookkeeper.feature.FeatureProvider;
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.feature.FeatureProvider;
/**
* Builder to build bookkeeper client.
@@ -187,10 +188,10 @@
}
private void validateParameters() {
- Preconditions.checkNotNull(name, "Missing client name.");
- Preconditions.checkNotNull(dlConfig, "Missing DistributedLog Configuration.");
- Preconditions.checkArgument(null == zkc || null == zkServers, "Missing zookeeper setting.");
- Preconditions.checkNotNull(ledgersPath, "Missing Ledgers Root Path.");
+ checkNotNull(name, "Missing client name.");
+ checkNotNull(dlConfig, "Missing DistributedLog Configuration.");
+ checkArgument(null == zkc || null == zkServers, "Missing zookeeper setting.");
+ checkNotNull(ledgersPath, "Missing Ledgers Root Path.");
}
public synchronized BookKeeperClient build() {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
index 613e60c..603859f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
@@ -17,21 +17,18 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Preconditions.checkArgument;
import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
import com.google.common.collect.Sets;
import java.net.URL;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.Set;
-import org.apache.distributedlog.bk.QuorumConfig;
-import org.apache.distributedlog.feature.DefaultFeatureProvider;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.net.DNSResolverForRacks;
-import org.apache.distributedlog.net.DNSResolverForRows;
+
import org.apache.bookkeeper.conf.ClientConfiguration;
import org.apache.bookkeeper.feature.FeatureProvider;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
+
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.util.ReflectionUtils;
import org.apache.commons.configuration.CompositeConfiguration;
@@ -40,26 +37,32 @@
import org.apache.commons.configuration.PropertiesConfiguration;
import org.apache.commons.configuration.SystemConfiguration;
import org.apache.commons.lang.StringUtils;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+
+
+import org.apache.distributedlog.bk.QuorumConfig;
+import org.apache.distributedlog.feature.DefaultFeatureProvider;
+import org.apache.distributedlog.net.DNSResolverForRacks;
+import org.apache.distributedlog.net.DNSResolverForRows;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* DistributedLog Configuration.
*
- * <p>
- * DistributedLog configuration is basically a properties based configuration, which extends from
+ * <p>DistributedLog configuration is basically a properties based configuration, which extends from
* Apache commons {@link CompositeConfiguration}. All the DL settings are in camel case and prefixed
* with a meaningful component name. for example, `zkSessionTimeoutSeconds` means <i>SessionTimeoutSeconds</i>
* for component `zk`.
*
- * <h3>BookKeeper Configuration</h3>
+ * <p></p><h3>BookKeeper Configuration</h3>
*
- * BookKeeper client configuration settings could be loaded via DistributedLog configuration. All those
+ * <p></p>BookKeeper client configuration settings could be loaded via DistributedLog configuration. All those
* settings are prefixed with <i>`bkc.`</i>. For example, <i>bkc.zkTimeout</i> in distributedlog configuration
* will be applied as <i>`zkTimeout`</i> in bookkeeper client configuration.
*
* <h3>How to load configuration</h3>
- *
* The default distributedlog configuration is constructed by instantiated a new instance. This
* distributedlog configuration will automatically load the settings that specified via
* {@link SystemConfiguration}.
@@ -67,7 +70,6 @@
* <pre>
* DistributedLogConfiguration conf = new DistributedLogConfiguration();
* </pre>
- *
* The recommended way is to load configuration from URL that points to a configuration file
* ({@link #loadConf(URL)}).
*
@@ -197,7 +199,8 @@
public static final String BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_OLD = "first-logsegment-sequence-number";
public static final long BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_DEFAULT =
DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO;
- public static final String BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED = "logSegmentSequenceNumberValidationEnabled";
+ public static final String BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED =
+ "logSegmentSequenceNumberValidationEnabled";
public static final boolean BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED_DEFAULT = true;
public static final String BKDL_ENABLE_RECORD_COUNTS = "enableRecordCounts";
public static final boolean BKDL_ENABLE_RECORD_COUNTS_DEFAULT = true;
@@ -230,7 +233,7 @@
public static final String BKDL_LOG_FLUSH_TIMEOUT = "logFlushTimeoutSeconds";
public static final int BKDL_LOG_FLUSH_TIMEOUT_DEFAULT = 30;
/**
- * CompressionCodec.Type String to use (See CompressionUtils)
+ * CompressionCodec.Type String to use (See CompressionUtils).
* --------------------- ------------------------------------
* NONE none
* LZ4 lz4
@@ -255,7 +258,8 @@
public static final int BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT = 0;
public static final String BKDL_ENABLE_IMMEDIATE_FLUSH = "enableImmediateFlush";
public static final boolean BKDL_ENABLE_IMMEDIATE_FLUSH_DEFAULT = false;
- public static final String BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS = "minimumDelayBetweenImmediateFlushMilliSeconds";
+ public static final String BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS =
+ "minimumDelayBetweenImmediateFlushMilliSeconds";
public static final int BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT = 0;
public static final String BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS = "periodicKeepAliveMilliSeconds";
public static final int BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS_DEFAULT = 0;
@@ -282,7 +286,8 @@
public static final String BKDL_LOCK_TIMEOUT = "lockTimeoutSeconds";
public static final long BKDL_LOCK_TIMEOUT_DEFAULT = 30;
public static final String BKDL_LOCK_REACQUIRE_TIMEOUT = "lockReacquireTimeoutSeconds";
- public static final long BKDL_LOCK_REACQUIRE_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_REACQUIRE_TIMEOUT_DEFAULT;
+ public static final long BKDL_LOCK_REACQUIRE_TIMEOUT_DEFAULT =
+ DistributedLogConstants.LOCK_REACQUIRE_TIMEOUT_DEFAULT;
public static final String BKDL_LOCK_OP_TIMEOUT = "lockOpTimeoutSeconds";
public static final long BKDL_LOCK_OP_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_OP_TIMEOUT_DEFAULT;
@@ -310,7 +315,8 @@
// General Settings
public static final String BKDL_READLAC_OPTION = "readLACLongPoll";
- public static final int BKDL_READLAC_OPTION_DEFAULT = 3; //BKLogPartitionReadHandler.ReadLACOption.READENTRYPIGGYBACK_SEQUENTIAL.value
+ public static final int BKDL_READLAC_OPTION_DEFAULT =
+ 3; //BKLogPartitionReadHandler.ReadLACOption.READENTRYPIGGYBACK_SEQUENTIAL.value
public static final String BKDL_READLACLONGPOLL_TIMEOUT = "readLACLongPollTimeout";
public static final int BKDL_READLACLONGPOLL_TIMEOUT_DEFAULT = 1000;
public static final String BKDL_DESERIALIZE_RECORDSET_ON_READS = "deserializeRecordSetOnReads";
@@ -358,7 +364,8 @@
public static final int BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT = 32;
// Scan Settings
- public static final String BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "firstNumEntriesEachPerLastRecordScan";
+ public static final String BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN =
+ "firstNumEntriesEachPerLastRecordScan";
public static final int BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 2;
public static final String BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "maxNumEntriesPerReadLastRecordScan";
public static final int BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 16;
@@ -376,15 +383,17 @@
public static final String BKDL_TRACE_READAHEAD_DELIVERY_LATENCY = "traceReadAheadDeliveryLatency";
public static final boolean BKDL_TRACE_READAHEAD_DELIVERY_LATENCY_DEFAULT = false;
public static final String BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS = "metadataLatencyWarnThresholdMs";
- public static final long BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
+ public static final long BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT =
+ DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
public static final String BKDL_DATA_LATENCY_WARN_THRESHOLD_MS = "dataLatencyWarnThresholdMs";
- public static final long BKDL_DATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = 2 * DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
+ public static final long BKDL_DATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT =
+ 2 * DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
public static final String BKDL_TRACE_READAHEAD_METADATA_CHANGES = "traceReadAheadMetadataChanges";
public static final boolean BKDL_TRACE_READAHEAD_MEATDATA_CHANGES_DEFAULT = false;
- public final static String BKDL_ENABLE_TASK_EXECUTION_STATS = "enableTaskExecutionStats";
- public final static boolean BKDL_ENABLE_TASK_EXECUTION_STATS_DEFAULT = false;
- public final static String BKDL_TASK_EXECUTION_WARN_TIME_MICROS = "taskExecutionWarnTimeMicros";
- public final static long BKDL_TASK_EXECUTION_WARN_TIME_MICROS_DEFAULT = 100000;
+ public static final String BKDL_ENABLE_TASK_EXECUTION_STATS = "enableTaskExecutionStats";
+ public static final boolean BKDL_ENABLE_TASK_EXECUTION_STATS_DEFAULT = false;
+ public static final String BKDL_TASK_EXECUTION_WARN_TIME_MICROS = "taskExecutionWarnTimeMicros";
+ public static final long BKDL_TASK_EXECUTION_WARN_TIME_MICROS_DEFAULT = 100000;
public static final String BKDL_ENABLE_PERSTREAM_STAT = "enablePerStreamStat";
public static final boolean BKDL_ENABLE_PERSTREAM_STAT_DEFAULT = false;
@@ -419,7 +428,8 @@
public static final String BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC = "dynamicConfigReloadIntervalSec";
public static final int BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC_DEFAULT = 60;
public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS = "streamConfigRouterClass";
- public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS_DEFAULT = "org.apache.distributedlog.service.config.IdentityConfigRouter";
+ public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS_DEFAULT =
+ "org.apache.distributedlog.service.config.IdentityConfigRouter";
// Settings for RateLimit (used by distributedlog-service)
@@ -495,7 +505,7 @@
}
/**
- * You can load configurations in precedence order. The first one takes
+ * You can load configurations in precedence order. The first one takes.
* precedence over any loaded later.
*
* @param confURL Configuration URL
@@ -506,7 +516,7 @@
}
/**
- * You can load configuration from other configuration
+ * You can load configuration from other configuration.
*
* @param baseConf Other Configuration
*/
@@ -515,7 +525,7 @@
}
/**
- * Load configuration from other configuration object
+ * Load configuration from other configuration object.
*
* @param otherConf Other configuration object
*/
@@ -524,7 +534,7 @@
}
/**
- * Load whitelisted stream configuration from another configuration object
+ * Load whitelisted stream configuration from another configuration object.
*
* @param streamConfiguration stream configuration overrides
* @Deprecated since 0.5.0, in favor of using {@link #loadStreamConf(java.util.Optional)}
@@ -580,7 +590,7 @@
if (appendNewline) {
builder.append("\n");
}
- Object value = getProperty((String)key);
+ Object value = getProperty((String) key);
builder.append(key).append("=").append(value);
appendNewline = true;
}
@@ -611,8 +621,8 @@
/**
* Get ZK Session timeout in seconds.
- * <p>
- * This is the session timeout applied for zookeeper client used by distributedlog.
+ *
+ * <p>This is the session timeout applied for zookeeper client used by distributedlog.
* Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
* by bookkeeper client.
*
@@ -625,8 +635,8 @@
/**
* Get ZK Session timeout in milliseconds.
- * <p>
- * This is the session timeout applied for zookeeper client used by distributedlog.
+ *
+ * <p>This is the session timeout applied for zookeeper client used by distributedlog.
* Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
* by bookkeeper client.
*
@@ -650,6 +660,7 @@
/**
* Get zookeeper access rate limit.
+ *
* <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
* It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
* the rate limiting is disable. By default it is disable (value = 0).
@@ -675,6 +686,7 @@
/**
* Get num of retries per request for zookeeper client.
+ *
* <p>Retries only happen on retryable failures like session expired,
* session moved. for permanent failures, the request will fail immediately.
* The default value is 3.
@@ -699,6 +711,7 @@
/**
* Get the start backoff time of zookeeper operation retries, in milliseconds.
+ *
* <p>The retry time will increase in bound exponential way, and become flat
* after hit max backoff time ({@link #getZKRetryBackoffMaxMillis()}).
* The default start backoff time is 5000 milliseconds.
@@ -726,6 +739,7 @@
/**
* Get the max backoff time of zookeeper operation retries, in milliseconds.
+ *
* <p>The retry time will increase in bound exponential way starting from
* {@link #getZKRetryBackoffStartMillis()}, and become flat after hit this max
* backoff time.
@@ -781,8 +795,8 @@
/**
* Get BK's zookeeper session timout in milliseconds.
- * <p>
- * This is the session timeout applied for zookeeper client used by bookkeeper client.
+ *
+ * <p>This is the session timeout applied for zookeeper client used by bookkeeper client.
* Use {@link #getZKSessionTimeoutMilliseconds()} for zookeeper client used
* by distributedlog.
*
@@ -806,6 +820,7 @@
/**
* Get zookeeper access rate limit for zookeeper client used in bookkeeper client.
+ *
* <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
* It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
* the rate limiting is disable. By default it is disable (value = 0).
@@ -833,6 +848,7 @@
/**
* Get num of retries for zookeeper client that used by bookkeeper client.
+ *
* <p>Retries only happen on retryable failures like session expired,
* session moved. for permanent failures, the request will fail immediately.
* The default value is 3. Setting it to zero or negative will retry infinitely.
@@ -849,6 +865,7 @@
/**
* Get the start backoff time of zookeeper operation retries, in milliseconds.
+ *
* <p>The retry time will increase in bound exponential way, and become flat
* after hit max backoff time ({@link #getBKClientZKRetryBackoffMaxMillis()}.
* The default start backoff time is 5000 milliseconds.
@@ -863,6 +880,7 @@
/**
* Get the max backoff time of zookeeper operation retries, in milliseconds.
+ *
* <p>The retry time will increase in bound exponential way starting from
* {@link #getBKClientZKRetryBackoffStartMillis()}, and become flat after
* hit this max backoff time.
@@ -883,14 +901,14 @@
/**
* Get ensemble size of each log segment (ledger) will use.
* By default it is 3.
- * <p>
- * A log segment's data is stored in an ensemble of bookies in
+ *
+ * <p>A log segment's data is stored in an ensemble of bookies in
* a stripping way. Each entry will be added in a <code>write-quorum</code>
* size of bookies. The add operation will complete once it receives
* responses from a <code>ack-quorum</code> size of bookies. The stripping
* is done in a round-robin way in bookkeeper.
- * <p>
- * For example, we configure the ensemble-size to 5, write-quorum-size to 3,
+ *
+ * <p>For example, we configure the ensemble-size to 5, write-quorum-size to 3,
* and ack-quorum-size to 2. The data will be stored in following stripping way.
* <pre>
* | entry id | bk1 | bk2 | bk3 | bk4 | bk5 |
@@ -901,8 +919,8 @@
* | 4 | x | x | | | x |
* | 5 | x | x | x | | |
* </pre>
- * <p>
- * We don't recommend stripping within a log segment to increase bandwidth.
+ *
+ * <p>We don't recommend stripping within a log segment to increase bandwidth.
* We'd recommend to strip by `partition` in higher level of distributedlog
* to increase performance. so typically the ensemble size will set to be
* the same value as write quorum size.
@@ -998,6 +1016,7 @@
/**
* Get if row aware ensemble placement is enabled.
+ *
* <p>If enabled, {@link DNSResolverForRows} will be used for dns resolution
* rather than {@link DNSResolverForRacks}, if no other dns resolver set via
* {@link #setEnsemblePlacementDnsResolverClass(Class)}.
@@ -1027,6 +1046,7 @@
/**
* Get the DNS resolver class for bookkeeper ensemble placement.
+ *
* <p>By default, {@link DNSResolverForRacks} will be used if
* {@link #getRowAwareEnsemblePlacementEnabled()} is disabled and
* {@link DNSResolverForRows} will be used if {@link #getRowAwareEnsemblePlacementEnabled()}
@@ -1064,6 +1084,7 @@
/**
* Get mapping used to override the region mapping derived by the default resolver.
+ *
* <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
* By default it is empty string.
*
@@ -1078,6 +1099,7 @@
/**
* Set mapping used to override the region mapping derived by the default resolver
+ *
* <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
* By default it is empty string.
*
@@ -1097,8 +1119,8 @@
/**
* Set password used by bookkeeper client for digestion.
- * <p>
- * NOTE: not recommend to change. will be derepcated in future.
+ *
+ * <p>NOTE: not recommend to change. will be derepcated in future.
*
* @param bkDigestPW BK password digest
* @return distributedlog configuration
@@ -1110,8 +1132,8 @@
/**
* Get password used by bookkeeper client for digestion.
- * <p>
- * NOTE: not recommend to change. will be deprecated in future.
+ *
+ * <p>NOTE: not recommend to change. will be deprecated in future.
*
* @return password used by bookkeeper client for digestion
* @see #setBKDigestPW(String)
@@ -1198,8 +1220,8 @@
/**
* Get BK client read timeout in seconds.
- * <p>
- * Please use {@link ClientConfiguration#getReadEntryTimeout()}
+ *
+ * <p>Please use {@link ClientConfiguration#getReadEntryTimeout()}
* instead of this setting.
*
* @return read timeout in seconds
@@ -1226,8 +1248,8 @@
/**
* Get BK client write timeout in seconds.
- * <p>
- * Please use {@link ClientConfiguration#getAddEntryTimeout()}
+ *
+ * <p>Please use {@link ClientConfiguration#getAddEntryTimeout()}
* instead of this setting.
*
* @return write timeout in seconds.
@@ -1239,7 +1261,7 @@
}
/**
- * Set BK client write timeout in seconds
+ * Set BK client write timeout in seconds.
*
* @param writeTimeout write timeout in seconds.
* @return distributed log configuration
@@ -1253,8 +1275,8 @@
/**
* Get BK client number of worker threads.
- * <p>
- * Please use {@link ClientConfiguration#getNumWorkerThreads()}
+ *
+ * <p>Please use {@link ClientConfiguration#getNumWorkerThreads()}
* instead of this setting.
*
* @return number of bookkeeper client worker threads.
@@ -1308,6 +1330,7 @@
/**
* Get the number of dedicated readahead worker threads used by distributedlog namespace.
+ *
* <p>If this value is non-positive, it would share the normal executor (see {@link #getNumWorkerThreads()}
* for readahead. otherwise, it would use a dedicated executor for readhead. By default,
* it is 0.
@@ -1362,8 +1385,8 @@
* By default it is 0 - the thread will be created dynamically by a executor service.
* The executor service is an unbounded pool. Application can use `total_tasks - completed_tasks`
* on monitoring the number of threads that are used for releasing resources.
- * <p>
- * The setting is only applied for v2 implementation.
+ *
+ * <p>The setting is only applied for v2 implementation.
*
* @return number of resource release threads used by distributedlog namespace.
*/
@@ -1456,8 +1479,8 @@
*/
public DistributedLogConfiguration setDLLedgerMetadataLayoutVersion(int layoutVersion)
throws IllegalArgumentException {
- if ((layoutVersion <= 0) ||
- (layoutVersion > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION)) {
+ if ((layoutVersion <= 0)
+ || (layoutVersion > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION)) {
// Incorrect version specified
throw new IllegalArgumentException("Incorrect value for ledger metadata layout version");
}
@@ -1479,6 +1502,7 @@
/**
* Set if we should skip the enforcement of min ledger metadata version.
+ *
* <p>NOTE: please be aware the side effects of skipping min ledger metadata
* version checking.
*
@@ -1486,7 +1510,8 @@
* @return distributed log configuration
* @see #getDLLedgerMetadataSkipMinVersionCheck()
*/
- public DistributedLogConfiguration setDLLedgerMetadataSkipMinVersionCheck(boolean skipMinVersionCheck) throws IllegalArgumentException {
+ public DistributedLogConfiguration setDLLedgerMetadataSkipMinVersionCheck(boolean skipMinVersionCheck)
+ throws IllegalArgumentException {
setProperty(BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK, skipMinVersionCheck);
return this;
}
@@ -1495,6 +1520,7 @@
* Get the value at which ledger sequence number should start for streams that are being
* upgraded and did not have ledger sequence number to start with or for newly created
* streams. By default, it is 1.
+ *
* <p>In most of the cases this value should not be changed. It is useful for backfilling
* in the case of migrating log segments whose metadata don't have log segment sequence number.
*
@@ -1508,8 +1534,7 @@
/**
* Set the value at which ledger sequence number should start for streams that are being
- * upgraded and did not have ledger sequence number to start with or for newly created
- * streams
+ *upgraded and did not have ledger sequence number to start with or for newly created streams.
*
* @param firstLogSegmentSequenceNumber first ledger sequence number
* @return distributed log configuration
@@ -1547,6 +1572,7 @@
/**
* Whether we should publish record counts in the log records and metadata.
+ *
* <p>By default it is true. This is a legacy setting for log segment version 1. It
* should be considered removed.
*
@@ -1570,6 +1596,7 @@
/**
* Whether sanity check txn id on starting log segments.
+ *
* <p>If it is enabled, DL writer would throw
* {@link org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException}
* when it received a smaller transaction id than current maximum transaction id.
@@ -1597,6 +1624,7 @@
/**
* Whether encode region id in log segment metadata.
+ *
* <p>In global DL use case, encoding region id in log segment medata would
* help understanding what region that a log segment is created. The region
* id field in log segment metadata would help for moniotring and troubleshooting.
@@ -1622,8 +1650,8 @@
/**
* Get log segment name version.
- * <p>
- * <ul>
+ *
+ * <p><ul>
* <li>version 0: inprogress_(start_txid) |
* logrecs_(start_txid)_(end_txid)</li>
* <li>version 1: inprogress_(logsegment_sequence_number) |
@@ -1652,6 +1680,7 @@
/**
* Get name of the unpartitioned stream.
+ *
* <p>It is a legacy setting. consider removing it in future.
*
* @return unpartitioned stream
@@ -1661,7 +1690,7 @@
}
/**
- * Set name of the unpartitioned stream
+ * Set name of the unpartitioned stream.
*
* @param streamName name of the unpartitioned stream
* @return distributedlog configuration
@@ -1764,6 +1793,7 @@
/**
* Get Log Flush timeout in seconds.
+ *
* <p>This is a setting used by DL writer on flushing data. It is typically used
* by synchronous writer and log segment writer. By default it is 30 seconds.
*
@@ -1804,13 +1834,14 @@
* @see #getCompressionType()
*/
public DistributedLogConfiguration setCompressionType(String compressionType) {
- Preconditions.checkArgument(null != compressionType && !compressionType.isEmpty());
+ checkArgument(null != compressionType && !compressionType.isEmpty());
setProperty(BKDL_COMPRESSION_TYPE, compressionType);
return this;
}
/**
* Whether to fail immediately if the stream is not ready rather than queueing the request.
+ *
* <p>If it is enabled, it would fail the write request immediately if the stream isn't ready.
* Consider turning it on for the use cases that could retry writing to other streams
* (aka non-strict ordering guarantee). It would result fast failure hence the client would
@@ -1866,6 +1897,7 @@
/**
* Check whether the durable write is enabled.
+ *
* <p>It is enabled by default.
*
* @return true if durable write is enabled. otherwise, false.
@@ -1892,10 +1924,12 @@
/**
* Get output buffer size for DL writers, in bytes.
+ *
* <p>Large buffer will result in higher compression ratio and
* it would use the bandwidth more efficiently and improve throughput.
* Set it to 0 would ask DL writers to transmit the data immediately,
* which it could achieve low latency.
+ *
* <p>The default value is 1KB.
*
* @return buffer size in byes.
@@ -1919,6 +1953,7 @@
/**
* Get Periodic Log Flush Frequency in milliseconds.
+ *
* <p>If the setting is set with a positive value, the data in output buffer
* will be flushed in this provided interval. The default value is 0.
*
@@ -1944,6 +1979,7 @@
/**
* Is immediate flush enabled.
+ *
* <p>If it is enabled, it would flush control record immediately after adding
* data completed. The default value is false.
*
@@ -1954,7 +1990,7 @@
}
/**
- * Enable/Disable immediate flush
+ * Enable/Disable immediate flush.
*
* @param enabled
* flag to enable/disable immediate flush.
@@ -1968,6 +2004,7 @@
/**
* Get minimum delay between immediate flushes in milliseconds.
+ *
* <p>This setting only takes effects when {@link #getImmediateFlushEnabled()}
* is enabled. It torelants the bursty of traffic when immediate flush is enabled,
* which prevents sending too many control records to the bookkeeper.
@@ -1976,11 +2013,12 @@
* @see #getImmediateFlushEnabled()
*/
public int getMinDelayBetweenImmediateFlushMs() {
- return this.getInt(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS, BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT);
+ return this.getInt(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS,
+ BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT);
}
/**
- * Set minimum delay between immediate flushes in milliseconds
+ * Set minimum delay between immediate flushes in milliseconds.
*
* @param minDelayMs minimum delay between immediate flushes in milliseconds.
* @return distributed log configuration
@@ -1993,6 +2031,7 @@
/**
* Get Periodic Keep Alive Frequency in milliseconds.
+ *
* <p>If the setting is set with a positive value, it would periodically write a control record
* to keep the stream active. The default value is 0.
*
@@ -2043,9 +2082,11 @@
/**
* Is truncation managed explicitly by the application.
+ *
* <p>If this is set then time based retention is only a hint to perform
* deferred cleanup. However we never remove a segment that has not been
* already marked truncated.
+ *
* <p>It is disabled by default.
*
* @return whether truncation managed explicitly by the application
@@ -2074,8 +2115,10 @@
/**
* Get log segment rolling interval in minutes.
+ *
* <p>If the setting is set to a positive value, DL writer will roll log segments
* based on time. Otherwise, it will roll log segments based on size.
+ *
* <p>The default value is 2 hours.
*
* @return log segment rolling interval in minutes
@@ -2101,9 +2144,11 @@
/**
* Get Max LogSegment Size in Bytes.
+ *
* <p>This setting only takes effects when time based rolling is disabled.
* DL writer will roll into a new log segment only after current one reaches
* this threshold.
+ *
* <p>The default value is 256MB.
*
* @return max logsegment size in bytes.
@@ -2132,6 +2177,7 @@
/**
* Get log segment rolling concurrency.
+ *
* <p>It limits how many writers could roll log segments concurrently.
* The default value is 1.
*
@@ -2163,7 +2209,8 @@
/**
* Is lock enabled when opening a writer to write a stream?
- * <p> We don't generally require a lock to write a stream to guarantee correctness. The lock
+ *
+ * <p>We don't generally require a lock to write a stream to guarantee correctness. The lock
* is more on tracking ownerships. The built-in fencing mechanism is used guarantee correctness
* during stream owner failover. It is okay to disable lock if your application knows which nodes
* have to write which streams.
@@ -2287,7 +2334,7 @@
}
/**
- * Set the root path of ledger allocator pool
+ * Set the root path of ledger allocator pool.
*
* @param path
* path of ledger allocator pool.
@@ -2349,6 +2396,7 @@
/**
* Get the per stream outstanding write limit for dl.
+ *
* <p>If the setting is set with a positive value, the per stream
* write limiting is enabled. By default it is disabled.
*
@@ -2375,6 +2423,7 @@
/**
* Get the global write limit for dl.
+ *
* <p>If the setting is set with a positive value, the global
* write limiting is enabled. By default it is disabled.
*
@@ -2400,8 +2449,10 @@
/**
* Whether to darkmode outstanding writes limit.
+ *
* <p>If it is running in darkmode, it would not reject requests when
* it is over limit, but just record them in the stats.
+ *
* <p>By default, it is in darkmode.
*
* @return flag to darmkode pending write limit.
@@ -2477,7 +2528,8 @@
//
/**
- * Get the time in milliseconds as the threshold for when an idle reader should dump warnings
+ * Get the time in milliseconds as the threshold for when an idle reader should dump warnings.
+ *
* <p>The default value is 2 minutes.
*
* @return reader idle warn threshold in millis.
@@ -2489,7 +2541,7 @@
}
/**
- * Set the time in milliseconds as the threshold for when an idle reader should dump warnings
+ * Set the time in milliseconds as the threshold for when an idle reader should dump warnings.
*
* @param warnThreshold time after which we should dump the read ahead state
* @return distributed log configuration
@@ -2501,7 +2553,8 @@
}
/**
- * Get the time in milliseconds as the threshold for when an idle reader should throw errors
+ * Get the time in milliseconds as the threshold for when an idle reader should throw errors.
+ *
* <p>The default value is <i>Integer.MAX_VALUE</i>.
*
* @return reader idle error threshold in millis
@@ -2513,7 +2566,7 @@
}
/**
- * Set the time in milliseconds as the threshold for when an idle reader should throw errors
+ * Set the time in milliseconds as the threshold for when an idle reader should throw errors.
*
* @param warnThreshold time after which we should throw idle reader errors
* @return distributed log configuration
@@ -2529,7 +2582,7 @@
//
/**
- * Get if we should ignore truncation status when reading the records
+ * Get if we should ignore truncation status when reading the records.
*
* @return if we should ignore truncation status
*/
@@ -2538,7 +2591,7 @@
}
/**
- * Set if we should ignore truncation status when reading the records
+ * Set if we should ignore truncation status when reading the records.
*
* @param ignoreTruncationStatus
* if we should ignore truncation status
@@ -2549,7 +2602,7 @@
}
/**
- * Get if we should alert when reader is positioned on a truncated segment
+ * Get if we should alert when reader is positioned on a truncated segment.
*
* @return if we should alert when reader is positioned on a truncated segment
*/
@@ -2558,7 +2611,7 @@
}
/**
- * Set if we should alert when reader is positioned on a truncated segment
+ * Set if we should alert when reader is positioned on a truncated segment.
*
* @param alertWhenPositioningOnTruncated
* if we should alert when reader is positioned on a truncated segment
@@ -2574,7 +2627,9 @@
* @return whether position gap detection for reader enabled.
*/
public boolean getPositionGapDetectionEnabled() {
- return getBoolean(BKDL_READER_POSITION_GAP_DETECTION_ENABLED, BKDL_READER_POSITION_GAP_DETECTION_ENABLED_DEFAULT);
+ return
+ getBoolean(BKDL_READER_POSITION_GAP_DETECTION_ENABLED,
+ BKDL_READER_POSITION_GAP_DETECTION_ENABLED_DEFAULT);
}
/**
@@ -2607,7 +2662,7 @@
}
/**
- * Get if we should enable read ahead
+ * Get if we should enable read ahead.
*
* @return if read ahead is enabled
*/
@@ -2616,7 +2671,7 @@
}
/**
- * Set if we should enable force read
+ * Set if we should enable force read.
*
* @param enableForceRead
* Enable force read
@@ -2627,7 +2682,7 @@
}
/**
- * Get if we should enable force read
+ * Get if we should enable force read.
*
* @return if should use separate ZK Clients
*/
@@ -2637,6 +2692,7 @@
/**
* Get the max records cached by readahead cache.
+ *
* <p>The default value is 10. Increase this value to improve throughput,
* but be careful about the memory.
*
@@ -2662,6 +2718,7 @@
/**
* Get number of entries read as a batch by readahead worker.
+ *
* <p>The default value is 2. Increase the value to increase the concurrency
* of reading entries from bookkeeper.
*
@@ -2698,7 +2755,7 @@
}
/**
- * Set the wait time between successive attempts to poll for new log records, in milliseconds
+ * Set the wait time between successive attempts to poll for new log records, in milliseconds.
*
* @param readAheadWaitTime read ahead wait time
* @return distributed log configuration
@@ -2712,6 +2769,7 @@
/**
* Get the wait time if it reaches end of stream and
* <b>there isn't any inprogress logsegment in the stream</b>, in millis.
+ *
* <p>The default value is 10 seconds.
*
* @see #setReadAheadWaitTimeOnEndOfStream(int)
@@ -2743,6 +2801,7 @@
* If readahead keeps receiving {@link org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException} on
* reading last add confirmed in given period, it would stop polling last add confirmed and re-initialize the ledger
* handle and retry. The threshold is specified in milliseconds.
+ *
* <p>The default value is 10 seconds.
*
* @return error threshold in milliseconds, that readahead will reinitialize ledger handle after keeping receiving
@@ -2754,7 +2813,8 @@
}
/**
- * Set the error threshold that readahead will reinitialize ledger handle after keeping receiving no such ledger exceptions.
+ * Set the error threshold that readahead will reinitialize ledger handle
+ * after keeping receiving no such ledger exceptions.
*
* @see #getReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis()
* @param thresholdMillis
@@ -2762,7 +2822,8 @@
* no such ledger exceptions.
* @return distributedlog configuration
*/
- public DistributedLogConfiguration setReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis(long thresholdMillis) {
+ public DistributedLogConfiguration
+ setReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis(long thresholdMillis) {
setProperty(BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS, thresholdMillis);
return this;
}
@@ -2839,7 +2900,8 @@
* @return number of entries to scan for first scan of reading last record.
*/
public int getFirstNumEntriesPerReadLastRecordScan() {
- return getInt(BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT);
+ return getInt(BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN,
+ BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT);
}
/**
@@ -2860,7 +2922,8 @@
* @return max number of entries for each scan to read last record.
*/
public int getMaxNumEntriesPerReadLastRecordScan() {
- return getInt(BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT);
+ return getInt(BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN,
+ BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT);
}
/**
@@ -3202,7 +3265,6 @@
/**
* Whether check the existence of a log if querying local cache of a federated namespace missed.
* Enabling it will issue zookeeper queries to check all sub namespaces under a federated namespace.
- *
* NOTE: by default it is on for all admin related tools. for write proxies, consider turning off for
* performance.
*
@@ -3316,6 +3378,7 @@
/**
* Get the maximum number of partitions of each stream allowed to be acquired per proxy.
+ *
* <p>This setting is able to configure per stream. This is the default setting if it is
* not configured per stream. Default value is -1, which means no limit on the number of
* partitions could be acquired each stream.
@@ -3341,6 +3404,7 @@
/**
* Get the maximum number of partitions of each stream allowed to cache per proxy.
+ *
* <p>This setting is able to configure per stream. This is the default setting if it is
* not configured per stream. Default value is -1, which means no limit on the number of
* partitions could be acquired each stream.
@@ -3524,15 +3588,15 @@
}
/**
- * Validate the configuration
+ * Validate the configuration.
*/
public void validate() {
- Preconditions.checkArgument(getBKClientReadTimeout() * 1000 >= getReadLACLongPollTimeout(),
- "Invalid timeout configuration: bkcReadTimeoutSeconds ("+getBKClientReadTimeout()+
- ") should be longer than readLACLongPollTimeout ("+getReadLACLongPollTimeout()+")");
+ checkArgument(getBKClientReadTimeout() * 1000 >= getReadLACLongPollTimeout(),
+ "Invalid timeout configuration: bkcReadTimeoutSeconds (" + getBKClientReadTimeout()
+ + ") should be longer than readLACLongPollTimeout (" + getReadLACLongPollTimeout() + ")");
long readerIdleWarnThresholdMs = getReaderIdleWarnThresholdMillis();
if (readerIdleWarnThresholdMs > 0) { // NOTE: some test cases set the idle warn threshold to 0
- Preconditions.checkArgument(readerIdleWarnThresholdMs > 2 * getReadLACLongPollTimeout(),
+ checkArgument(readerIdleWarnThresholdMs > 2 * getReadLACLongPollTimeout(),
"Invalid configuration: ReaderIdleWarnThreshold should be 2x larget than readLACLongPollTimeout");
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
index 4688f40..d8bdc3d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
@@ -17,14 +17,13 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Charsets.UTF_8;
+import com.google.common.collect.ImmutableList;
import java.util.List;
import java.util.concurrent.TimeUnit;
-
-import com.google.common.collect.ImmutableList;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.ACL;
-import static com.google.common.base.Charsets.UTF_8;
/**
* Constants used in DistributedLog.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java b/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
index 1a4583d..2a44a5e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
@@ -17,14 +17,14 @@
*/
package org.apache.distributedlog;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkNotNull;
import io.netty.buffer.ByteBuf;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.distributedlog.exceptions.LogRecordTooLongException;
import org.apache.distributedlog.exceptions.WriteException;
import org.apache.distributedlog.io.CompressionCodec;
-import org.apache.bookkeeper.stats.NullStatsLogger;
/**
* A set of {@link LogRecord}s.
@@ -139,7 +139,7 @@
}
public Entry.Reader buildReader() throws IOException {
- Preconditions.checkNotNull(buffer,
+ checkNotNull(buffer,
"Serialized data or input stream isn't provided");
return new EnvelopedEntryReader(
logSegmentSequenceNumber,
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
index 89eb6e9..0e17929 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
@@ -27,29 +27,28 @@
/**
* An enveloped entry written to BookKeeper.
*
- * Data type in brackets. Interpretation should be on the basis of data types and not individual
+ * <p>Data type in brackets. Interpretation should be on the basis of data types and not individual
* bytes to honor Endianness.
*
- * Entry Structure:
+ * <p>Entry Structure:
* ---------------
* Bytes 0 : Version (Byte)
* Bytes 1 - (DATA = 1+Header.length-1) : Header (Integer)
* Bytes DATA - DATA+3 : Payload Length (Integer)
* BYTES DATA+4 - DATA+4+payload.length-1 : Payload (Byte[])
*
- * V1 Header Structure: // Offsets relative to the start of the header.
+ * <p>V1 Header Structure: // Offsets relative to the start of the header.
* -------------------
* Bytes 0 - 3 : Flags (Integer)
* Bytes 4 - 7 : Original payload size before compression (Integer)
*
- * Flags: // 32 Bits
+ * <p>Flags: // 32 Bits
* -----
* 0 ... 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
* |_|
* |
* Compression Type
- *
- * Compression Type: // 2 Bits (Least significant)
+ *Compression Type: // 2 Bits (Least significant)
* ----------------
* 00 : No Compression
* 01 : LZ4 Compression
@@ -76,8 +75,7 @@
/**
* Return an {@link ByteBuf} that reads from the provided {@link ByteBuf}, decompresses the data
* and returns a new InputStream wrapping the underlying payload.
- *
- * Note that src is modified by this call.
+ *Note that src is modified by this call.
*
* @return
* New Input stream with the underlying payload.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java
index 3924f4b..82656ba 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java
@@ -119,7 +119,7 @@
if (isExhausted) {
return;
}
- releaseBuffer();;
+ releaseBuffer();
}
@Override
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
index 86cc56e..98c88f7 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
@@ -49,7 +49,7 @@
*/
class EnvelopedEntryWriter implements Writer {
- static final Logger logger = LoggerFactory.getLogger(EnvelopedEntryWriter.class);
+ private static final Logger logger = LoggerFactory.getLogger(EnvelopedEntryWriter.class);
private static class WriteRequest {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
index d897274..ff7e953 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
@@ -18,20 +18,6 @@
package org.apache.distributedlog;
import com.google.common.base.Optional;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.metadata.DLMetadata;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.bookkeeper.util.LocalBookKeeper;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.net.BindException;
@@ -40,10 +26,25 @@
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.bookkeeper.util.LocalBookKeeper;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
/**
* Utility class for setting up bookkeeper ensembles
- * and bringing individual bookies up and down
+ * and bringing individual bookies up and down.
*/
public class LocalDLMEmulator {
private static final Logger LOG = LoggerFactory.getLogger(LocalDLMEmulator.class);
@@ -67,6 +68,9 @@
private final int zkPort;
private final int numBookies;
+ /**
+ * Builder to build LocalDLMEmulator.
+ */
public static class Builder {
private int zkTimeoutSec = DEFAULT_ZK_TIMEOUT_SEC;
private int numBookies = DEFAULT_NUM_BOOKIES;
@@ -126,7 +130,9 @@
return new Builder();
}
- private LocalDLMEmulator(final int numBookies, final boolean shouldStartZK, final String zkHost, final int zkPort, final int initialBookiePort, final int zkTimeoutSec, final ServerConfiguration serverConf) throws Exception {
+ private LocalDLMEmulator(final int numBookies, final boolean shouldStartZK,
+ final String zkHost, final int zkPort, final int initialBookiePort,
+ final int zkTimeoutSec, final ServerConfiguration serverConf) throws Exception {
this.numBookies = numBookies;
this.zkHost = zkHost;
this.zkPort = zkPort;
@@ -137,7 +143,8 @@
public void run() {
try {
LOG.info("Starting {} bookies : allowLoopback = {}", numBookies, serverConf.getAllowLoopback());
- LocalBookKeeper.startLocalBookies(zkHost, zkPort, numBookies, shouldStartZK, initialBookiePort, serverConf);
+ LocalBookKeeper.startLocalBookies(zkHost, zkPort,
+ numBookies, shouldStartZK, initialBookiePort, serverConf);
LOG.info("{} bookies are started.");
} catch (InterruptedException e) {
// go away quietly
@@ -150,7 +157,7 @@
public void start() throws Exception {
bkStartupThread.start();
- if (!LocalBookKeeper.waitForServerUp(zkEnsemble, zkTimeoutSec*1000)) {
+ if (!LocalBookKeeper.waitForServerUp(zkEnsemble, zkTimeoutSec * 1000)) {
throw new Exception("Error starting zookeeper/bookkeeper");
}
int bookiesUp = checkBookiesUp(numBookies, zkTimeoutSec);
@@ -182,7 +189,7 @@
}
/**
- * Check that a number of bookies are available
+ * Check that a number of bookies are available.
*
* @param count number of bookies required
* @param timeout number of seconds to wait for bookies to start
@@ -260,7 +267,7 @@
* Try to start zookkeeper locally on any port.
*/
public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(File zkDir) throws Exception {
- return runZookeeperOnAnyPort((int) (Math.random()*10000+7000), zkDir);
+ return runZookeeperOnAnyPort((int) (Math.random() * 10000 + 7000), zkDir);
}
/**
@@ -269,9 +276,9 @@
*/
public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(int basePort, File zkDir) throws Exception {
- final int MAX_RETRIES = 20;
- final int MIN_PORT = 1025;
- final int MAX_PORT = 65535;
+ final int maxRetries = 20;
+ final int minPort = 1025;
+ final int maxPort = 65535;
ZooKeeperServerShim zks = null;
int zkPort = basePort;
boolean success = false;
@@ -284,12 +291,12 @@
success = true;
} catch (BindException be) {
retries++;
- if (retries > MAX_RETRIES) {
+ if (retries > maxRetries) {
throw be;
}
zkPort++;
- if (zkPort > MAX_PORT) {
- zkPort = MIN_PORT;
+ if (zkPort > maxPort) {
+ zkPort = minPort;
}
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
index 462ddaa..c8907b5 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
@@ -17,18 +17,18 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Charsets.UTF_8;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
import java.io.File;
import java.io.IOException;
import java.util.Comparator;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
import org.apache.distributedlog.exceptions.UnsupportedMetadataVersionException;
import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.Utils;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
@@ -37,16 +37,17 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static com.google.common.base.Charsets.UTF_8;
/**
* Utility class for storing the metadata associated
- * with a single edit log segment, stored in a single ledger
+ * with a single edit log segment, stored in a single ledger.
*/
public class LogSegmentMetadata {
static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadata.class);
-
- public static enum LogSegmentMetadataVersion {
+ /**
+ * LogSegmentMetadataVersion.
+ */
+ public enum LogSegmentMetadataVersion {
VERSION_INVALID(0),
VERSION_V1_ORIGINAL(1),
VERSION_V2_LEDGER_SEQNO(2),
@@ -79,8 +80,10 @@
}
}
}
-
- public static enum TruncationStatus {
+ /**
+ * TruncationStatus.
+ */
+ public enum TruncationStatus {
ACTIVE (0), PARTIALLY_TRUNCATED(1), TRUNCATED (2);
private final int value;
@@ -89,6 +92,9 @@
}
}
+ /**
+ * LogSegmentMetadataBuilder to build LogSegmentMetadata.
+ */
public static class LogSegmentMetadataBuilder {
protected String zkPath;
protected long logSegmentId;
@@ -327,13 +333,13 @@
// NOTE: This value is not stored in the Metadata store.
private final boolean envelopeEntries;
- public static final Comparator<LogSegmentMetadata> COMPARATOR
- = new Comparator<LogSegmentMetadata>() {
+ public static final Comparator<LogSegmentMetadata> COMPARATOR =
+ new Comparator<LogSegmentMetadata>() {
public int compare(LogSegmentMetadata o1,
LogSegmentMetadata o2) {
- if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
- (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
+ if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)
+ || (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
if (o1.firstTxId < o2.firstTxId) {
return -1;
} else if (o1.firstTxId == o2.firstTxId) {
@@ -362,12 +368,12 @@
}
};
- public static final Comparator<LogSegmentMetadata> DESC_COMPARATOR
- = new Comparator<LogSegmentMetadata>() {
+ public static final Comparator<LogSegmentMetadata> DESC_COMPARATOR =
+ new Comparator<LogSegmentMetadata>() {
public int compare(LogSegmentMetadata o1,
LogSegmentMetadata o2) {
- if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
- (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
+ if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)
+ || (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
if (o1.firstTxId > o2.firstTxId) {
return -1;
} else if (o1.firstTxId == o2.firstTxId) {
@@ -493,8 +499,8 @@
public long getStartSequenceId() {
// generate negative sequence id for log segments that created <= v4
- return supportsSequenceId() && startSequenceId != DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ?
- startSequenceId : Long.MIN_VALUE + (getLogSegmentSequenceNumber() << 32L);
+ return supportsSequenceId() && startSequenceId != DistributedLogConstants.UNASSIGNED_SEQUENCE_ID
+ ? startSequenceId : Long.MIN_VALUE + (getLogSegmentSequenceNumber() << 32L);
}
public boolean isTruncated() {
@@ -593,7 +599,8 @@
return read(zkc, path, false);
}
- public static CompletableFuture<LogSegmentMetadata> read(ZooKeeperClient zkc, String path, final boolean skipMinVersionCheck) {
+ public static CompletableFuture<LogSegmentMetadata> read(ZooKeeperClient zkc,
+ String path, final boolean skipMinVersionCheck) {
final CompletableFuture<LogSegmentMetadata> result = new CompletableFuture<LogSegmentMetadata>();
try {
zkc.get().getData(path, false, new AsyncCallback.DataCallback() {
@@ -636,7 +643,7 @@
LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V1_ORIGINAL;
- int regionId = (int)(versionStatusCount & REGION_MASK) >> REGION_SHIFT;
+ int regionId = (int) (versionStatusCount & REGION_MASK) >> REGION_SHIFT;
assert (regionId >= 0 && regionId <= 0xf);
long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
@@ -681,7 +688,7 @@
LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO;
- int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
+ int regionId = (int) ((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
assert (regionId >= 0 && regionId <= 0xf);
long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
@@ -731,12 +738,12 @@
long version = versionStatusCount & METADATA_VERSION_MASK;
assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
- assert (LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version &&
- LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version);
+ assert (LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version
+ && LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version);
LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
- int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
+ int regionId = (int) ((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
assert (regionId >= 0 && regionId <= 0xf);
long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
@@ -801,12 +808,12 @@
long version = versionStatusCount & METADATA_VERSION_MASK;
assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
- assert (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version &&
- LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version);
+ assert (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version
+ && LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version);
LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
- int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
+ int regionId = (int) ((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
assert (regionId >= 0 && regionId <= 0xf);
long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
@@ -880,21 +887,21 @@
}
if (!skipMinVersionCheck && version < LogSegmentMetadata.LEDGER_METADATA_OLDEST_SUPPORTED_VERSION) {
- throw new UnsupportedMetadataVersionException("Ledger metadata version '" + version + "' is no longer supported: "
- + new String(data, UTF_8));
+ throw new UnsupportedMetadataVersionException("Ledger metadata version '"
+ + version + "' is no longer supported: " + new String(data, UTF_8));
}
if (version > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION) {
- throw new UnsupportedMetadataVersionException("Metadata version '" + version + "' is higher than the highest supported version : "
- + new String(data, UTF_8));
+ throw new UnsupportedMetadataVersionException("Metadata version '"
+ + version + "' is higher than the highest supported version : " + new String(data, UTF_8));
}
if (LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value == version) {
return parseDataV1(path, data, parts);
} else if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value == version) {
return parseDataV2(path, data, parts);
- } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version &&
- LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version) {
+ } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version
+ && LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version) {
return parseDataVersionsWithMinActiveDLSN(path, data, parts);
} else {
assert(version >= LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
@@ -919,7 +926,7 @@
finalisedData = String.format("%d;%d;%d",
version.value, logSegmentId, firstTxId);
} else {
- long versionAndCount = ((long) version.value) | ((long)recordCount << LOGRECORD_COUNT_SHIFT);
+ long versionAndCount = ((long) version.value) | ((long) recordCount << LOGRECORD_COUNT_SHIFT);
finalisedData = String.format("%d;%d;%d;%d;%d",
versionAndCount, logSegmentId, firstTxId, lastTxId, completionTime);
}
@@ -928,7 +935,7 @@
versionStatusCount |= ((status & METADATA_STATUS_BIT_MAX) << STATUS_BITS_SHIFT);
versionStatusCount |= (((long) regionId & MAX_REGION_ID) << REGION_SHIFT);
if (!inprogress) {
- versionStatusCount |= ((long)recordCount << LOGRECORD_COUNT_SHIFT);
+ versionStatusCount |= ((long) recordCount << LOGRECORD_COUNT_SHIFT);
}
if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO == version) {
if (inprogress) {
@@ -939,21 +946,23 @@
versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
logSegmentSeqNo, lastEntryId, lastSlotId);
}
- } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version.value &&
- LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version.value) {
+ } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version.value
+ && LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version.value) {
if (inprogress) {
finalisedData = String.format("%d;%d;%d;%d;%d;%d",
- versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId);
+ versionStatusCount, logSegmentId, firstTxId,
+ logSegmentSeqNo, minActiveEntryId, minActiveSlotId);
} else {
finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
logSegmentSeqNo, lastEntryId, lastSlotId, minActiveEntryId, minActiveSlotId);
}
- } else if (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version.value &&
- LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version.value) {
+ } else if (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version.value
+ && LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version.value) {
if (inprogress) {
finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d",
- versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId, startSequenceId);
+ versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo,
+ minActiveEntryId, minActiveSlotId, startSequenceId);
} else {
finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
@@ -1004,9 +1013,9 @@
// completionTime is set when a node is finalized, so that
// cannot be compared
// if the node is inprogress, don't compare the lastTxId either
- if (this.getLogSegmentSequenceNumber() != other.getLogSegmentSequenceNumber() ||
- this.logSegmentId != other.logSegmentId ||
- this.firstTxId != other.firstTxId) {
+ if (this.getLogSegmentSequenceNumber() != other.getLogSegmentSequenceNumber()
+ || this.logSegmentId != other.logSegmentId
+ || this.firstTxId != other.firstTxId) {
retVal = false;
} else if (this.inprogress) {
retVal = other.inprogress;
@@ -1054,21 +1063,21 @@
}
public String toString() {
- return "[LogSegmentId:" + logSegmentId +
- ", firstTxId:" + firstTxId +
- ", lastTxId:" + lastTxId +
- ", version:" + version +
- ", completionTime:" + completionTime +
- ", recordCount:" + recordCount +
- ", regionId:" + regionId +
- ", status:" + status +
- ", logSegmentSequenceNumber:" + getLogSegmentSequenceNumber() +
- ", lastEntryId:" + getLastEntryId() +
- ", lastSlotId:" + getLastSlotId() +
- ", inprogress:" + inprogress +
- ", minActiveDLSN:" + minActiveDLSN +
- ", startSequenceId:" + startSequenceId +
- "]";
+ return "[LogSegmentId:" + logSegmentId
+ + ", firstTxId:" + firstTxId
+ + ", lastTxId:" + lastTxId
+ + ", version:" + version
+ + ", completionTime:" + completionTime
+ + ", recordCount:" + recordCount
+ + ", regionId:" + regionId
+ + ", status:" + status
+ + ", logSegmentSequenceNumber:" + getLogSegmentSequenceNumber()
+ + ", lastEntryId:" + getLastEntryId()
+ + ", lastSlotId:" + getLastSlotId()
+ + ", inprogress:" + inprogress
+ + ", minActiveDLSN:" + minActiveDLSN
+ + ", startSequenceId:" + startSequenceId
+ + "]";
}
public Mutator mutator() {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java
index a76f547..047f44a 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java
@@ -17,12 +17,13 @@
*/
package org.apache.distributedlog;
-import org.apache.distributedlog.util.DLUtils;
import org.apache.bookkeeper.versioning.Version;
import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.distributedlog.util.DLUtils;
+
/**
- * Utility class for storing and reading max ledger sequence number
+ * Utility class for storing and reading max ledger sequence number.
*/
class MaxLogSegmentSequenceNo {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java
index 8f077e2..bcc41d1 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java
@@ -17,15 +17,15 @@
*/
package org.apache.distributedlog;
-import org.apache.distributedlog.util.DLUtils;
import org.apache.bookkeeper.versioning.Version;
import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.distributedlog.util.DLUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Utility class for storing and reading
- * the max seen txid in zookeeper
+ * the max seen txid in zookeeper.
*/
class MaxTxId {
static final Logger LOG = LoggerFactory.getLogger(MaxTxId.class);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
index 8183703..4a49fdf 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
@@ -21,10 +21,23 @@
import com.google.common.base.Stopwatch;
import com.google.common.base.Ticker;
import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.versioning.Versioned;
import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
import org.apache.distributedlog.exceptions.DLIllegalStateException;
import org.apache.distributedlog.exceptions.DLInterruptedException;
@@ -35,29 +48,17 @@
import org.apache.distributedlog.logsegment.LogSegmentEntryReader;
import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
import org.apache.distributedlog.logsegment.LogSegmentFilter;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.versioning.Versioned;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
+
/**
* New ReadAhead Reader that uses {@link org.apache.distributedlog.logsegment.LogSegmentEntryReader}.
*
- * NOTE: all the state changes happen in the same thread. All *unsafe* methods should be submitted to the order
- * scheduler using stream name as the key.
+ * <p>NOTE: all the state changes happen in the same thread. All *unsafe* methods should be submitted to the order
+ * scheduler using stream name as the key.</p>
*/
class ReadAheadEntryReader implements
AsyncCloseable,
@@ -158,7 +159,7 @@
}
@Override
- synchronized public void onSuccess(LogSegmentEntryReader reader) {
+ public synchronized void onSuccess(LogSegmentEntryReader reader) {
this.reader = reader;
if (reader.getSegment().isInProgress()) {
reader.registerListener(ReadAheadEntryReader.this);
@@ -723,7 +724,7 @@
}
/**
- * Reinitialize the log segments
+ * Reinitialize the log segments.
*/
private void unsafeReinitializeLogSegments(List<LogSegmentMetadata> segments) {
logger.info("Reinitialize log segments with {}", segments);
@@ -807,15 +808,15 @@
continue;
}
// if the log segment is truncated, skip it.
- if (skipTruncatedLogSegments &&
- !conf.getIgnoreTruncationStatus() &&
- segment.isTruncated()) {
+ if (skipTruncatedLogSegments
+ && !conf.getIgnoreTruncationStatus()
+ && segment.isTruncated()) {
continue;
}
// if the log segment is partially truncated, move the start dlsn to the min active dlsn
- if (skipTruncatedLogSegments &&
- !conf.getIgnoreTruncationStatus() &&
- segment.isPartiallyTruncated()) {
+ if (skipTruncatedLogSegments
+ && !conf.getIgnoreTruncationStatus()
+ && segment.isPartiallyTruncated()) {
if (segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
dlsnToStart = segment.getMinActiveDLSN();
}
@@ -877,8 +878,8 @@
+ " on a segment " + segment + " that is already marked as truncated"));
return false;
}
- if (segment.isPartiallyTruncated() &&
- segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
+ if (segment.isPartiallyTruncated()
+ && segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
if (conf.getAlertWhenPositioningOnTruncated()) {
alertStatsLogger.raise("Trying to position reader on {} when {} is marked partially truncated",
fromDLSN, segment);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
index bf4e140..29684a3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
@@ -17,6 +17,8 @@
*/
package org.apache.distributedlog;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CompletableFuture;
@@ -24,21 +26,20 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
import org.apache.distributedlog.selector.FirstDLSNNotLessThanSelector;
import org.apache.distributedlog.selector.FirstTxIdNotLessThanSelector;
import org.apache.distributedlog.selector.LastRecordSelector;
import org.apache.distributedlog.selector.LogRecordSelector;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
- * Utility function for readers
+ * Utility function for readers.
*/
public class ReadUtils {
@@ -516,7 +517,7 @@
long lastEntryId = reader.getLastAddConfirmed();
if (lastEntryId < 0) {
// it means that the log segment is created but not written yet or an empty log segment.
- // it is equivalent to 'all log records whose transaction id is less than provided transactionId'
+ //it is equivalent to 'all log records whose transaction id is less than provided transactionId'
Optional<LogRecordWithDLSN> nonRecord = Optional.absent();
promise.complete(nonRecord);
return;
@@ -641,7 +642,7 @@
}
/**
- * Process the search results
+ * Process the search results.
*/
static void processSearchResults(
final String logName,
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
index 04bb9e4..1c97bd8 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
@@ -17,8 +17,8 @@
*/
package org.apache.distributedlog;
-import org.apache.distributedlog.exceptions.OverCapacityException;
import org.apache.distributedlog.common.util.PermitLimiter;
+import org.apache.distributedlog.exceptions.OverCapacityException;
class WriteLimiter {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
index e56a22d..8f074eb 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
@@ -17,24 +17,8 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Charsets.UTF_8;
import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.zk.ZKWatcherManager;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.Watcher.Event.EventType;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.ACL;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashSet;
@@ -42,9 +26,25 @@
import java.util.Set;
import java.util.concurrent.CopyOnWriteArraySet;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.zk.ZKWatcherManager;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
-import static com.google.common.base.Charsets.UTF_8;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
/**
* ZooKeeper Client wrapper over {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}.
@@ -61,7 +61,10 @@
*/
public class ZooKeeperClient {
- public static interface Credentials {
+ /**
+ * interface used to authenticate zk client.
+ */
+ public interface Credentials {
Credentials NONE = new Credentials() {
@Override
@@ -72,7 +75,9 @@
void authenticate(ZooKeeper zooKeeper);
}
-
+ /**
+ * interface impl used to authenticate zk client.
+ */
public static class DigestCredentials implements Credentials {
String username;
@@ -89,6 +94,9 @@
}
}
+ /**
+ * Notify a zk session expire event.
+ */
public interface ZooKeeperSessionExpireNotifier {
void notifySessionExpired();
}
@@ -193,7 +201,6 @@
* @return a connected ZooKeeper client
* @throws ZooKeeperConnectionException if there was a problem connecting to the ZK cluster
* @throws InterruptedException if interrupted while waiting for a connection to be established
- * @throws TimeoutException if a connection could not be established within the configured
* session timeout
*/
public synchronized ZooKeeper get()
@@ -202,7 +209,8 @@
try {
FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
} catch (IOException ioe) {
- throw new ZooKeeperConnectionException("Client " + name + " failed on establishing zookeeper connection", ioe);
+ throw new ZooKeeperConnectionException("Client "
+ + name + " failed on establishing zookeeper connection", ioe);
}
// This indicates that the client was explictly closed
@@ -260,8 +268,8 @@
break;
case Disconnected:
if (null == retryPolicy) {
- LOG.info("ZooKeeper {} is disconnected from zookeeper now," +
- " but it is OK unless we received EXPIRED event.", name);
+ LOG.info("ZooKeeper {} is disconnected from zookeeper now,"
+ + " but it is OK unless we received EXPIRED event.", name);
}
// Mark as not authenticated if expired or disconnected. In both cases
// we lose any attached auth info. Relying on Expired/Disconnected is
@@ -294,10 +302,10 @@
ZooKeeper zk;
try {
- RetryPolicy opRetryPolicy = null == retryPolicy ?
- new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) : retryPolicy;
- RetryPolicy connectRetryPolicy = null == retryPolicy ?
- new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) :
+ RetryPolicy opRetryPolicy = null == retryPolicy
+ ? new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) : retryPolicy;
+ RetryPolicy connectRetryPolicy = null == retryPolicy
+ ? new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) :
new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, Integer.MAX_VALUE);
zk = org.apache.bookkeeper.zookeeper.ZooKeeperClient.newBuilder()
.connectString(zooKeeperServers)
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
index 0c200ce..9e2ec96 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
@@ -17,18 +17,20 @@
*/
package org.apache.distributedlog;
-import com.google.common.base.Preconditions;
-import org.apache.distributedlog.ZooKeeperClient.Credentials;
-import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
-import org.apache.distributedlog.impl.BKNamespaceDriver;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import java.net.URI;
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.distributedlog.ZooKeeperClient.Credentials;
+import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.net.URI;
/**
* Builder to build zookeeper client.
@@ -72,7 +74,7 @@
private ZooKeeperClientBuilder() {}
/**
- * Set zookeeper client name
+ * Set zookeeper client name.
*
* @param name zookeeper client name
* @return zookeeper client builder
@@ -108,7 +110,7 @@
}
/**
- * Set zookeeper connection timeout in milliseconds
+ * Set zookeeper connection timeout in milliseconds.
*
* @param connectionTimeoutMs
* connection timeout ms.
@@ -189,13 +191,13 @@
}
private void validateParameters() {
- Preconditions.checkNotNull(zkServers, "No zk servers provided.");
- Preconditions.checkArgument(conectionTimeoutMs > 0,
+ checkNotNull(zkServers, "No zk servers provided.");
+ checkArgument(conectionTimeoutMs > 0,
"Invalid connection timeout : %d", conectionTimeoutMs);
- Preconditions.checkArgument(sessionTimeoutMs > 0,
+ checkArgument(sessionTimeoutMs > 0,
"Invalid session timeout : %d", sessionTimeoutMs);
- Preconditions.checkNotNull(statsLogger, "No stats logger provided.");
- Preconditions.checkArgument(zkAclIdSet, "Zookeeper acl id not set.");
+ checkNotNull(statsLogger, "No stats logger provided.");
+ checkArgument(zkAclIdSet, "Zookeeper acl id not set.");
}
/**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java
index 2c3e738..3d634f9 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java
@@ -18,7 +18,7 @@
package org.apache.distributedlog.acl;
/**
- * Access Control on stream operations
+ * Access Control on stream operations.
*/
public interface AccessControlManager {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java
index bf3352a..e86cd0c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java
@@ -17,6 +17,9 @@
*/
package org.apache.distributedlog.acl;
+/**
+ * Default Access Control.
+ */
public class DefaultAccessControlManager implements AccessControlManager {
public static final DefaultAccessControlManager INSTANCE = new DefaultAccessControlManager();
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
index c4939c0..1183869 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
@@ -17,40 +17,8 @@
*/
package org.apache.distributedlog.admin;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkArgument;
import com.google.common.collect.Lists;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Function;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.LogRecordWithDLSN;
-import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.ReadUtils;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.ZooKeeperClientBuilder;
-import org.apache.distributedlog.api.namespace.Namespace;
-import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.impl.acl.ZKAccessControl;
-import org.apache.distributedlog.exceptions.DLIllegalStateException;
-import org.apache.distributedlog.impl.federated.FederatedZKLogMetadataStore;
-import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.metadata.DLMetadata;
-import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.metadata.MetadataUpdater;
-import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.thrift.AccessControlEntry;
-import org.apache.distributedlog.tools.DistributedLogTool;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.common.util.SchedulerUtils;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URI;
@@ -63,13 +31,55 @@
import java.util.Map;
import java.util.SortedSet;
import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CountDownLatch;
+
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.ReadUtils;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.namespace.Namespace;
+
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.SchedulerUtils;
+
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+
+
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.impl.acl.ZKAccessControl;
+import org.apache.distributedlog.impl.federated.FederatedZKLogMetadataStore;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
+import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
+
+import org.apache.distributedlog.metadata.MetadataUpdater;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.thrift.AccessControlEntry;
+import org.apache.distributedlog.tools.DistributedLogTool;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
/**
* Admin Tool for DistributedLog.
@@ -113,16 +123,19 @@
LogSegmentMetadata inprogressSegment = null;
for (LogSegmentMetadata segment : segments) {
if (!segment.isInProgress()) {
- maxCompletedLogSegmentSequenceNumber = Math.max(maxCompletedLogSegmentSequenceNumber, segment.getLogSegmentSequenceNumber());
+ maxCompletedLogSegmentSequenceNumber = Math.max(maxCompletedLogSegmentSequenceNumber,
+ segment.getLogSegmentSequenceNumber());
} else {
// we already found an inprogress segment
if (null != inprogressSegment) {
- throw new DLIllegalStateException("Multiple inprogress segments found for stream " + streamName + " : " + segments);
+ throw new DLIllegalStateException("Multiple inprogress segments found for stream "
+ + streamName + " : " + segments);
}
inprogressSegment = segment;
}
}
- if (null == inprogressSegment || inprogressSegment.getLogSegmentSequenceNumber() > maxCompletedLogSegmentSequenceNumber) {
+ if (null == inprogressSegment || inprogressSegment.getLogSegmentSequenceNumber()
+ > maxCompletedLogSegmentSequenceNumber) {
// nothing to fix
return;
}
@@ -131,7 +144,8 @@
return;
}
final LogSegmentMetadata newSegment =
- FutureUtils.result(metadataUpdater.changeSequenceNumber(inprogressSegment, newLogSegmentSequenceNumber));
+ FutureUtils.result(metadataUpdater.changeSequenceNumber(inprogressSegment,
+ newLogSegmentSequenceNumber));
LOG.info("Fixed {} : {} -> {} ",
new Object[] { streamName, inprogressSegment, newSegment });
if (verbose) {
@@ -206,7 +220,7 @@
final boolean verbose,
final boolean interactive,
final int concurrency) throws Exception {
- Preconditions.checkArgument(concurrency > 0, "Invalid concurrency " + concurrency + " found.");
+ checkArgument(concurrency > 0, "Invalid concurrency " + concurrency + " found.");
// 0. getting streams under a given uri.
Iterator<String> streamsIter = namespace.getLogs();
List<String> streams = Lists.newArrayList();
@@ -225,7 +239,8 @@
if (verbose) {
System.out.println("+ 0. " + streamCandidates.size() + " corrupted streams found.");
}
- if (interactive && !IOUtils.confirmPrompt("Do you want to fix all " + streamCandidates.size() + " corrupted streams (Y/N) : ")) {
+ if (interactive && !IOUtils.confirmPrompt("Do you want to fix all "
+ + streamCandidates.size() + " corrupted streams (Y/N) : ")) {
return;
}
if (verbose) {
@@ -371,10 +386,10 @@
).thenApply(new Function<LogRecordWithDLSN, LogSegmentCandidate>() {
@Override
public LogSegmentCandidate apply(LogRecordWithDLSN record) {
- if (null != record &&
- (record.getDlsn().compareTo(metadata.getLastDLSN()) > 0 ||
- record.getTransactionId() > metadata.getLastTxId() ||
- !metadata.isRecordPositionWithinSegmentScope(record))) {
+ if (null != record
+ && (record.getDlsn().compareTo(metadata.getLastDLSN()) > 0
+ || record.getTransactionId() > metadata.getLastTxId()
+ || !metadata.isRecordPositionWithinSegmentScope(record))) {
return new LogSegmentCandidate(metadata, record);
} else {
return null;
@@ -396,14 +411,15 @@
}
System.out.println("-------------------------------------------");
}
- if (interactive && !IOUtils.confirmPrompt("Do you want to fix the stream " + streamCandidate.streamName + " (Y/N) : ")) {
+ if (interactive && !IOUtils.confirmPrompt("Do you want to fix the stream "
+ + streamCandidate.streamName + " (Y/N) : ")) {
return false;
}
for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
LogSegmentMetadata newMetadata = FutureUtils.result(
metadataUpdater.updateLastRecord(segmentCandidate.metadata, segmentCandidate.lastRecord));
if (verbose) {
- System.out.println(" Fixed segment " + segmentCandidate.metadata.getLogSegmentSequenceNumber() + " : ");
+ System.out.println(" Fixed segment " + segmentCandidate.metadata.getLogSegmentSequenceNumber() + " : ");
System.out.println(" old metadata : " + segmentCandidate.metadata);
System.out.println(" new metadata : " + newMetadata);
}
@@ -420,7 +436,6 @@
/**
* Unbind the bookkeeper environment for a given distributedlog uri.
- *
* TODO: move unbind operation to namespace driver
*/
class UnbindCommand extends OptsCommand {
@@ -481,7 +496,6 @@
/**
* Bind Command to bind bookkeeper environment for a given distributed uri.
- *
* TODO: move bind to namespace driver
*/
class BindCommand extends OptsCommand {
@@ -492,12 +506,16 @@
super("bind", "bind the bookkeeper environment settings for a given distributedlog instance.");
options.addOption("l", "bkLedgers", true, "ZooKeeper ledgers path for bookkeeper instance.");
options.addOption("s", "bkZkServers", true, "ZooKeeper servers used for bookkeeper for writers.");
- options.addOption("bkzr", "bkZkServersForReader", true, "ZooKeeper servers used for bookkeeper for readers.");
- options.addOption("dlzw", "dlZkServersForWriter", true, "ZooKeeper servers used for distributedlog for writers.");
- options.addOption("dlzr", "dlZkServersForReader", true, "ZooKeeper servers used for distributedlog for readers.");
+ options.addOption("bkzr", "bkZkServersForReader", true,
+ "ZooKeeper servers used for bookkeeper for readers.");
+ options.addOption("dlzw", "dlZkServersForWriter", true,
+ "ZooKeeper servers used for distributedlog for writers.");
+ options.addOption("dlzr", "dlZkServersForReader", true,
+ "ZooKeeper servers used for distributedlog for readers.");
options.addOption("i", "sanityCheckTxnID", true, "Flag to sanity check highest txn id.");
options.addOption("r", "encodeRegionID", true, "Flag to encode region id.");
- options.addOption("seqno", "firstLogSegmentSeqNo", true, "The first log segment sequence number to use after upgrade");
+ options.addOption("seqno", "firstLogSegmentSeqNo", true,
+ "The first log segment sequence number to use after upgrade");
options.addOption("fns", "federatedNamespace", false, "Flag to turn a namespace to federated namespace");
options.addOption("f", "force", false, "Force binding without prompt.");
options.addOption("c", "creation", false, "Whether is it a creation binding.");
@@ -570,7 +588,8 @@
.setEncodeRegionID(encodeRegionID);
if (cmdline.hasOption("seqno")) {
- newBKDLConfig = newBKDLConfig.setFirstLogSegmentSeqNo(Long.parseLong(cmdline.getOptionValue("seqno")));
+ newBKDLConfig = newBKDLConfig.setFirstLogSegmentSeqNo(
+ Long.parseLong(cmdline.getOptionValue("seqno")));
}
if (cmdline.hasOption("fns")) {
@@ -594,7 +613,7 @@
if (newBKDLConfig.equals(bkdlConfig)) {
System.out.println("No bookkeeper binding needs to be updated. Quit.");
return 0;
- } else if(!newBKDLConfig.isFederatedNamespace() && bkdlConfig.isFederatedNamespace()) {
+ } else if (!newBKDLConfig.isFederatedNamespace() && bkdlConfig.isFederatedNamespace()) {
System.out.println("You can't turn a federated namespace back to non-federated.");
return 0;
} else {
@@ -670,8 +689,8 @@
@Override
protected int runCmd() throws Exception {
- MetadataUpdater metadataUpdater = dryrun ?
- new DryrunLogSegmentMetadataStoreUpdater(getConf(),
+ MetadataUpdater metadataUpdater = dryrun
+ ? new DryrunLogSegmentMetadataStoreUpdater(getConf(),
getLogSegmentMetadataStore()) :
LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
getLogSegmentMetadataStore());
@@ -681,7 +700,8 @@
return -1;
}
for (String stream : streams) {
- fixInprogressSegmentWithLowerSequenceNumber(getNamespace(), metadataUpdater, stream, verbose, !getForce());
+ fixInprogressSegmentWithLowerSequenceNumber(getNamespace(),
+ metadataUpdater, stream, verbose, !getForce());
}
return 0;
}
@@ -721,8 +741,8 @@
@Override
protected int runCmd() throws Exception {
- MetadataUpdater metadataUpdater = dryrun ?
- new DryrunLogSegmentMetadataStoreUpdater(getConf(),
+ MetadataUpdater metadataUpdater = dryrun
+ ? new DryrunLogSegmentMetadataStoreUpdater(getConf(),
getLogSegmentMetadataStore()) :
LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
getLogSegmentMetadataStore());
@@ -829,7 +849,7 @@
}
}
- static abstract class SetACLCommand extends PerDLCommand {
+ abstract static class SetACLCommand extends PerDLCommand {
boolean denyWrite = false;
boolean denyTruncate = false;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java
index d708111..80b7e77 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * Admin Tools for DistributedLog
+ * Admin Tools for DistributedLog.
*/
package org.apache.distributedlog.admin;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogReader.java
index 217236e..022b7bc 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogReader.java
@@ -25,8 +25,12 @@
import org.apache.distributedlog.LogRecordWithDLSN;
import org.apache.distributedlog.io.AsyncCloseable;
+/**
+ * AsyncLogReader.
+ */
@Public
@Evolving
+
public interface AsyncLogReader extends AsyncCloseable {
/**
@@ -34,14 +38,14 @@
*
* @return stream name.
*/
- public String getStreamName();
+ String getStreamName();
/**
- * Read the next record from the log stream
+ * Read the next record from the log stream.
*
* @return A promise that when satisfied will contain the Log Record with its DLSN.
*/
- public CompletableFuture<LogRecordWithDLSN> readNext();
+ CompletableFuture<LogRecordWithDLSN> readNext();
/**
* Read next <i>numEntries</i> entries. The future is only satisfied with non-empty list
@@ -52,12 +56,12 @@
* num entries
* @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
*/
- public CompletableFuture<List<LogRecordWithDLSN>> readBulk(int numEntries);
+ CompletableFuture<List<LogRecordWithDLSN>> readBulk(int numEntries);
/**
* Read next <i>numEntries</i> entries in a given <i>waitTime</i>.
- * <p>
- * The future is satisfied when either reads <i>numEntries</i> entries or reaches <i>waitTime</i>.
+ *
+ * <p>The future is satisfied when either reads <i>numEntries</i> entries or reaches <i>waitTime</i>.
* The only exception is if there isn't any new entries written within <i>waitTime</i>, it would
* wait until new entries are available.
*
@@ -69,5 +73,5 @@
* wait time unit
* @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
*/
- public CompletableFuture<List<LogRecordWithDLSN>> readBulk(int numEntries, long waitTime, TimeUnit timeUnit);
+ CompletableFuture<List<LogRecordWithDLSN>> readBulk(int numEntries, long waitTime, TimeUnit timeUnit);
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogWriter.java
index 8bb45a2..9c8d1b9 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/AsyncLogWriter.java
@@ -26,6 +26,9 @@
import org.apache.distributedlog.io.AsyncAbortable;
import org.apache.distributedlog.io.AsyncCloseable;
+/**
+ * AsyncLogWriter.
+ */
@Public
@Evolving
public interface AsyncLogWriter extends AsyncCloseable, AsyncAbortable {
@@ -76,7 +79,7 @@
CompletableFuture<Long> markEndOfStream();
/**
- * Get the name of the stream this writer writes data to
+ * Get the name of the stream this writer writes data to.
*/
String getStreamName();
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java
index 46f8b35..fb9c6be 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/DistributedLogManager.java
@@ -23,15 +23,17 @@
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public;
import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving;
+
import org.apache.distributedlog.AppendOnlyStreamReader;
import org.apache.distributedlog.AppendOnlyStreamWriter;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.LogRecordWithDLSN;
import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
+
import org.apache.distributedlog.callback.LogSegmentListener;
import org.apache.distributedlog.io.AsyncCloseable;
import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.api.subscription.SubscriptionsStore;
/**
* A DistributedLogManager is responsible for managing a single place of storing
@@ -45,17 +47,17 @@
public interface DistributedLogManager extends AsyncCloseable, Closeable {
/**
- * Get the name of the stream managed by this log manager
+ * Get the name of the stream managed by this log manager.
* @return streamName
*/
- public String getStreamName();
+ String getStreamName();
/**
* Get the namespace driver used by this manager.
*
* @return the namespace driver
*/
- public NamespaceDriver getNamespaceDriver();
+ NamespaceDriver getNamespaceDriver();
//
// Log Segment Related Operations
@@ -67,7 +69,7 @@
* @return log segments
* @throws IOException
*/
- public List<LogSegmentMetadata> getLogSegments() throws IOException;
+ List<LogSegmentMetadata> getLogSegments() throws IOException;
/**
* Register <i>listener</i> on log segment updates of this stream.
@@ -75,7 +77,7 @@
* @param listener
* listener to receive update log segment list.
*/
- public void registerListener(LogSegmentListener listener) throws IOException ;
+ void registerListener(LogSegmentListener listener) throws IOException;
/**
* Unregister <i>listener</i> on log segment updates from this stream.
@@ -83,7 +85,7 @@
* @param listener
* listener to receive update log segment list.
*/
- public void unregisterListener(LogSegmentListener listener);
+ void unregisterListener(LogSegmentListener listener);
//
// Writer & Reader Operations
@@ -94,7 +96,7 @@
*
* @return result represents the open result
*/
- public CompletableFuture<AsyncLogWriter> openAsyncLogWriter();
+ CompletableFuture<AsyncLogWriter> openAsyncLogWriter();
/**
* Open sync log writer to write records to the log stream.
@@ -102,23 +104,23 @@
* @return sync log writer
* @throws IOException when fails to open a sync log writer.
*/
- public LogWriter openLogWriter() throws IOException;
+ LogWriter openLogWriter() throws IOException;
/**
- * Begin writing to the log stream identified by the name
+ * Begin writing to the log stream identified by the name.
*
* @return the writer interface to generate log records
* @Deprecated since 0.5.0, in favor of using {@link #openLogWriter()}
*/
- public LogWriter startLogSegmentNonPartitioned() throws IOException;
+ LogWriter startLogSegmentNonPartitioned() throws IOException;
/**
- * Begin writing to the log stream identified by the name
+ * Begin writing to the log stream identified by the name.
*
* @return the writer interface to generate log records
* @Deprecated since 0.5.0, in favor of using {@link #openAsyncLogWriter()}
*/
- public AsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException;
+ AsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException;
/**
* Open an sync log reader to read records from a log starting from <code>fromTxnId</code>.
@@ -127,16 +129,16 @@
* transaction id to start reading from
* @return sync log reader
*/
- public LogReader openLogReader(long fromTxnId) throws IOException;
+ LogReader openLogReader(long fromTxnId) throws IOException;
/**
- * Open an async log reader to read records from a log starting from <code>fromDLSN</code>
+ * Open an async log reader to read records from a log starting from <code>fromDLSN</code>.
*
* @param fromDLSN
* dlsn to start reading from
* @return async log reader
*/
- public LogReader openLogReader(DLSN fromDLSN) throws IOException;
+ LogReader openLogReader(DLSN fromDLSN) throws IOException;
/**
* Open an async log reader to read records from a log starting from <code>fromTxnId</code>.
@@ -145,37 +147,37 @@
* transaction id to start reading from
* @return async log reader
*/
- public CompletableFuture<AsyncLogReader> openAsyncLogReader(long fromTxnId);
+ CompletableFuture<AsyncLogReader> openAsyncLogReader(long fromTxnId);
/**
- * Open an async log reader to read records from a log starting from <code>fromDLSN</code>
+ * Open an async log reader to read records from a log starting from <code>fromDLSN</code>.
*
* @param fromDLSN
* dlsn to start reading from
* @return async log reader
*/
- public CompletableFuture<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN);
+ CompletableFuture<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN);
/**
- * Get the input stream starting with fromTxnId for the specified log
+ * Get the input stream starting with fromTxnId for the specified log.
*
* @param fromTxnId - the first transaction id we want to read
* @return the stream starting with transaction fromTxnId
* @throws IOException if a stream cannot be found.
* @Deprecated since 0.5.0, in favor of using {@link #openLogReader(long)}
*/
- public LogReader getInputStream(long fromTxnId)
+ LogReader getInputStream(long fromTxnId)
throws IOException;
/**
- * Get the input stream starting with fromTxnId for the specified log
+ * Get the input stream starting with fromTxnId for the specified log.
*
* @param fromDLSN - the first DLSN we want to read
* @return the stream starting with DLSN
* @throws IOException if a stream cannot be found.
* @Deprecated since 0.5.0, in favor of using {@link #openLogReader(DLSN)}
*/
- public LogReader getInputStream(DLSN fromDLSN) throws IOException;
+ LogReader getInputStream(DLSN fromDLSN) throws IOException;
/**
* Get an async log reader to read records from a log starting from <code>fromTxnId</code>.
@@ -187,7 +189,7 @@
* @see #openAsyncLogReader(long)
* @Deprecated it is deprecated since 0.5.0, in favor of using {@link #openAsyncLogReader(long)}
*/
- public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException;
+ AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException;
/**
* Get an async log reader to read records from a log starting from <code>fromDLSN</code>.
@@ -199,7 +201,7 @@
* @see #openAsyncLogReader(DLSN)
* @Deprecated it is deprecated since 0.5.0, in favor of using {@link #openAsyncLogReader(DLSN)}
*/
- public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException;
+ AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException;
/**
* Get a log reader with lock starting from <i>fromDLSN</i>.
@@ -211,7 +213,7 @@
* start dlsn
* @return async log reader
*/
- public CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
+ CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
/**
* Get a log reader with lock starting from <i>fromDLSN</i> and using <i>subscriberId</i>.
@@ -225,7 +227,7 @@
* subscriber id
* @return async log reader
*/
- public CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN, String subscriberId);
+ CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN, String subscriberId);
/**
* Get a log reader using <i>subscriberId</i> with lock. The reader will start reading from
@@ -239,25 +241,25 @@
* subscriber id
* @return async log reader
*/
- public CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
+ CompletableFuture<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
//
// Stream writer and reader
//
/**
- * Begin appending to the end of the log stream which is being treated as a sequence of bytes
+ * Begin appending to the end of the log stream which is being treated as a sequence of bytes.
*
* @return the writer interface to generate log records
*/
- public AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException;
+ AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException;
/**
- * Get a reader to read a log stream as a sequence of bytes
+ * Get a reader to read a log stream as a sequence of bytes.
*
* @return the writer interface to generate log records
*/
- public AppendOnlyStreamReader getAppendOnlyStreamReader() throws IOException;
+ AppendOnlyStreamReader getAppendOnlyStreamReader() throws IOException;
//
// Metadata Operations:
@@ -275,77 +277,77 @@
* transaction id
* @return dlsn of first log record whose transaction id is not less than transactionId.
*/
- public CompletableFuture<DLSN> getDLSNNotLessThanTxId(long transactionId);
+ CompletableFuture<DLSN> getDLSNNotLessThanTxId(long transactionId);
/**
- * Get the last log record in the stream
+ * Get the last log record in the stream.
*
* @return the last log record in the stream
* @throws IOException if a stream cannot be found.
*/
- public LogRecordWithDLSN getLastLogRecord()
+ LogRecordWithDLSN getLastLogRecord()
throws IOException;
/**
- * Get Latest log record with DLSN in the log - async
+ * Get Latest log record with DLSN in the log - async.
*
* @return latest log record with DLSN
*/
- public CompletableFuture<LogRecordWithDLSN> getLastLogRecordAsync();
+ CompletableFuture<LogRecordWithDLSN> getLastLogRecordAsync();
/**
- * Get the earliest Transaction Id available in the log
+ * Get the earliest Transaction Id available in the log.
*
* @return earliest transaction id
* @throws IOException
*/
- public long getFirstTxId() throws IOException;
+ long getFirstTxId() throws IOException;
/**
- * Get Latest Transaction Id in the log
+ * Get Latest Transaction Id in the log.
*
* @return latest transaction id
* @throws IOException
*/
- public long getLastTxId() throws IOException;
+ long getLastTxId() throws IOException;
/**
- * Get Latest Transaction Id in the log - async
+ * Get Latest Transaction Id in the log - async.
*
* @return latest transaction id
*/
- public CompletableFuture<Long> getLastTxIdAsync();
+ CompletableFuture<Long> getLastTxIdAsync();
/**
* Get first DLSN in the log.
*
* @return first dlsn in the stream
*/
- public CompletableFuture<DLSN> getFirstDLSNAsync();
+ CompletableFuture<DLSN> getFirstDLSNAsync();
/**
- * Get Latest DLSN in the log
+ * Get Latest DLSN in the log.
*
* @return last dlsn
* @throws IOException
*/
- public DLSN getLastDLSN() throws IOException;
+ DLSN getLastDLSN() throws IOException;
/**
- * Get Latest DLSN in the log - async
+ * Get Latest DLSN in the log - async.
*
* @return latest transaction id
*/
- public CompletableFuture<DLSN> getLastDLSNAsync();
+ CompletableFuture<DLSN> getLastDLSNAsync();
/**
- * Get the number of log records in the active portion of the log
+ * 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
*/
- public long getLogRecordCount() throws IOException;
+ long getLogRecordCount() throws IOException;
/**
* Get the number of log records in the active portion of the log - async.
@@ -354,23 +356,23 @@
* @return future number of log records
* @throws IOException
*/
- public CompletableFuture<Long> getLogRecordCountAsync(final DLSN beginDLSN);
+ CompletableFuture<Long> getLogRecordCountAsync(DLSN beginDLSN);
/**
* Run recovery on the log.
*
* @throws IOException
*/
- public void recover() throws IOException;
+ void recover() throws IOException;
/**
- * Check if an end of stream marker was added to the stream
+ * 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
* @throws IOException
*/
- public boolean isEndOfStreamMarked() throws IOException;
+ boolean isEndOfStreamMarked() throws IOException;
/**
* Delete the log.
@@ -379,7 +381,7 @@
* @Deprecated since 0.5.0, in favor of using
* {@link org.apache.distributedlog.api.namespace.Namespace#deleteLog(String)}
*/
- public void delete() throws IOException;
+ void delete() throws IOException;
/**
* The DistributedLogManager may archive/purge any logs for transactionId
@@ -391,13 +393,13 @@
* @param minTxIdToKeep the earliest txid that must be retained
* @throws IOException if purging fails
*/
- public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException;
+ void purgeLogsOlderThan(long minTxIdToKeep) throws IOException;
/**
* Get the subscriptions store provided by the distributedlog manager.
*
* @return subscriptions store manages subscriptions for current stream.
*/
- public SubscriptionsStore getSubscriptionsStore();
+ SubscriptionsStore getSubscriptionsStore();
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogReader.java
index eb1ef9c..6ee3370 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogReader.java
@@ -31,22 +31,21 @@
* <i>LogReader</i> is a `synchronous` reader reading records from a DL log.
*
* <h3>Lifecycle of a Reader</h3>
- *
* A reader is a <i>sequential</i> reader that read records from a DL log starting
* from a given position. The position could be a <i>DLSN</i> (via {@link DistributedLogManager#getInputStream(DLSN)}
* or a <i>Transaction ID</i> (via {@link DistributedLogManager#getInputStream(long)}.
- * <p>
- * After the reader is open, it could call {@link #readNext(boolean)} or {@link #readBulk(boolean, int)}
+ *
+ * <p>After the reader is open, it could call {@link #readNext(boolean)} or {@link #readBulk(boolean, int)}
* to read records out the log from provided position.
- * <p>
- * Closing the reader (via {@link #close()} will release all the resources occupied
+ *
+ * <p>Closing the reader (via {@link #close()} will release all the resources occupied
* by this reader instance.
- * <p>
- * Exceptions could be thrown during reading records. Once the exception is thrown,
+ *
+ * <p>Exceptions could be thrown during reading records. Once the exception is thrown,
* the reader is set to an error state and it isn't usable anymore. It is the application's
* responsibility to handle the exceptions and re-create readers if necessary.
- * <p>
- * Example:
+ *
+ * <p>Example:
* <pre>
* DistributedLogManager dlm = ...;
* long nextTxId = ...;
@@ -69,7 +68,6 @@
* </pre>
*
* <h3>Read Records</h3>
- *
* Reading records from an <i>endless</i> log in `synchronous` way isn't as
* trivial as in `asynchronous` way (via {@link AsyncLogReader}. Because it
* lacks of callback mechanism. LogReader introduces a flag `nonBlocking` on
@@ -81,8 +79,8 @@
* before returning read calls. While <i>NonBlocking</i> (nonBlocking = true)
* means the reads will only check readahead cache and return whatever records
* available in the readahead cache.
- * <p>
- * The <i>waiting</i> period varies in <i>blocking</i> mode. If the reader is
+ *
+ * <p>The <i>waiting</i> period varies in <i>blocking</i> mode. If the reader is
* catching up with writer (there are records in the log), the read call will
* wait until records are read and returned. If the reader is caught up with
* writer (there are no more records in the log at read time), the read call
@@ -91,14 +89,14 @@
* records available in the readahead cache. In other words, if a reader sees
* no record on blocking reads, it means the reader is `caught-up` with the
* writer.
- * <p>
- * <i>Blocking</i> and <i>NonBlocking</i> modes are useful for building replicated
+ *
+ * <p><i>Blocking</i> and <i>NonBlocking</i> modes are useful for building replicated
* state machines. Applications could use <i>blocking</i> reads till caught up
* with latest data. Once they are caught up with latest data, they could start
* serving their service and turn to <i>non-blocking</i> read mode and tail read
* data from the logs.
- * <p>
- * See examples below.
+ *
+ * <p>See examples below.
*
* <h4>Read Single Record</h4>
*
@@ -135,8 +133,8 @@
* LogReader reader = ...
* int N = 10;
*
- * // keep reading N records in blocking way until no records available in the log
- * List<LogRecord> records = reader.readBulk(false, N);
+ *<p>// keep reading N records in blocking way until no records available in the log
+ * List<LogRecord> records = reader.readBulk(false, N);
* while (!records.isEmpty()) {
* // process the list of records
* ...
@@ -155,11 +153,10 @@
* // process the new records
* ...
* }
- *
+ *</p>
* </pre>
*
- * <p>
- * NOTE: Extending {@link AsyncCloseable}: BKSyncLogReader is implemented based on BKAsyncLogReader, exposing
+ * <p>NOTE: Extending {@link AsyncCloseable}: BKSyncLogReader is implemented based on BKAsyncLogReader, exposing
* the {@link AsyncCloseable} interface so the reader could be closed asynchronously
*
* @see AsyncLogReader
@@ -170,12 +167,12 @@
/**
* Read the next log record from the stream.
- * <p>
- * If <i>nonBlocking</i> is set to true, the call returns immediately by just polling
+ *
+ * <p>If <i>nonBlocking</i> is set to true, the call returns immediately by just polling
* records from read ahead cache. It would return <i>null</i> if there isn't any records
* available in the read ahead cache.
- * <p>
- * If <i>nonBlocking</i> is set to false, it would does blocking call. The call will
+ *
+ * <p>If <i>nonBlocking</i> is set to false, it would does blocking call. The call will
* block until return a record if there are records in the stream (aka catching up).
* Otherwise it would wait up to {@link DistributedLogConfiguration#getReadAheadWaitTime()}
* milliseconds and return null if there isn't any more records in the stream.
@@ -185,10 +182,10 @@
* @return an operation from the stream or null if at end of stream
* @throws IOException if there is an error reading from the stream
*/
- public LogRecordWithDLSN readNext(boolean nonBlocking) throws IOException;
+ LogRecordWithDLSN readNext(boolean nonBlocking) throws IOException;
/**
- * Read the next <i>numLogRecords</i> log records from the stream
+ * Read the next <i>numLogRecords</i> log records from the stream.
*
* @param nonBlocking should the read make blocking calls to the backend or rely on the
* readAhead cache
@@ -197,5 +194,5 @@
* @throws IOException if there is an error reading from the stream
* @see #readNext(boolean)
*/
- public List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords) throws IOException;
+ List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords) throws IOException;
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogWriter.java
index e72b368..8afd590 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/LogWriter.java
@@ -25,10 +25,11 @@
import org.apache.distributedlog.LogRecord;
import org.apache.distributedlog.io.Abortable;
-/*
-* A generic interface class to support writing log records into
-* a persistent distributed log.
-*/
+
+/**
+ * A generic interface class to support writing log records into
+ * a persistent distributed log.
+ */
@Public
@Evolving
public interface LogWriter extends Closeable, Abortable {
@@ -38,7 +39,7 @@
* @param record single log record
* @throws IOException
*/
- public void write(LogRecord record) throws IOException;
+ void write(LogRecord record) throws IOException;
/**
@@ -48,7 +49,7 @@
* @throws IOException
*/
@Deprecated
- public int writeBulk(List<LogRecord> records) throws IOException;
+ int writeBulk(List<LogRecord> records) throws IOException;
/**
* All data that has been written to the stream so far will be sent to
@@ -56,14 +57,14 @@
* The transmission is asynchronous and new data can be still written to the
* stream while flushing is performed.
*/
- public long flush() throws IOException;
+ long flush() throws IOException;
/**
* Flush and sync all data that is ready to be flush
* {@link #flush()} into underlying persistent store.
* @throws IOException
*/
- public long commit() throws IOException;
+ long commit() throws IOException;
/**
* Flushes all the data up to this point,
@@ -73,6 +74,6 @@
*
* @throws IOException
*/
- public void markEndOfStream() throws IOException;
+ void markEndOfStream() throws IOException;
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/MetadataAccessor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/MetadataAccessor.java
index d73c99b..a940676 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/MetadataAccessor.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/MetadataAccessor.java
@@ -26,27 +26,27 @@
/**
* Provide a metadata accessor to access customized metadata associated with logs.
*
- * @Deprecated this class is here for legacy reason. It is not recommended to use this class for storing customized
+ * @deprecated this class is here for legacy reason. It is not recommended to use this class for storing customized
* metadata.
*/
@LimitedPrivate
@Evolving
public interface MetadataAccessor extends Closeable, AsyncCloseable {
/**
- * Get the name of the stream managed by this log manager
+ * Get the name of the stream managed by this log manager.
* @return streamName
*/
- public String getStreamName();
+ String getStreamName();
- public void createOrUpdateMetadata(byte[] metadata) throws IOException;
+ void createOrUpdateMetadata(byte[] metadata) throws IOException;
- public void deleteMetadata() throws IOException;
+ void deleteMetadata() throws IOException;
- public byte[] getMetadata() throws IOException;
+ byte[] getMetadata() throws IOException;
/**
* Close the distributed log metadata, freeing any resources it may hold.
*/
- public void close() throws IOException;
+ void close() throws IOException;
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java
index fc3629f..cad0f97 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/Namespace.java
@@ -28,8 +28,8 @@
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.callback.NamespaceListener;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.LogNotFoundException;
import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
import org.apache.distributedlog.namespace.NamespaceDriver;
/**
@@ -49,20 +49,21 @@
*
* <h4>Namespace Location</h4>
*
- * At the highest level, a <code>Namespace</code> is located by a <code>URI</code>. The location
- * URI is in string form has the syntax
+ * <p>At the highest level, a <code>Namespace</code> is located by a <code>URI</code>. The location
+ * URI is in string form has the syntax</p>
*
* <blockquote>
* distributedlog[<tt><b>-</b></tt><i>provider</i>]<tt><b>:</b></tt><i>provider-specific-path</i>
* </blockquote>
*
- * where square brackets [...] delineate optional components and the characters <tt><b>-</b></tt> and <tt><b>:</b></tt>
- * stand for themselves.
+ * <p>where square brackets [...] delineate optional components and the characters
+ * <tt><b>-</b></tt> and <tt><b>:</b></tt>
+ * stand for themselves.</p>
*
- * The <code>provider</code> part in the URI indicates what is the backend used for this namespace. For example:
+ * <p>The <code>provider</code> part in the URI indicates what is the backend used for this namespace. For example:
* <i>distributedlog-bk</i> URI is storing logs in bookkeeper, while <i>distributedlog-mem</i> URI is storing logs in
* memory. The <code>provider</code> part is optional. It would use bookkeeper backend if the <i>provider</i> part
- * is omitted.
+ * is omitted.</p>
*
* @see DistributedLogManager
* @since 0.3.32
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java
index 6b01731..c555031 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/namespace/NamespaceBuilder.java
@@ -17,14 +17,21 @@
*/
package org.apache.distributedlog.api.namespace;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkNotNull;
import java.io.IOException;
import java.net.URI;
+
import org.apache.bookkeeper.common.annotation.InterfaceAudience.Public;
import org.apache.bookkeeper.common.annotation.InterfaceStability.Stable;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.feature.SettableFeatureProvider;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.distributedlog.BKDistributedLogNamespace;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.common.util.PermitLimiter;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.feature.CoreFeatureKeys;
import org.apache.distributedlog.injector.AsyncFailureInjector;
@@ -34,13 +41,8 @@
import org.apache.distributedlog.util.ConfUtils;
import org.apache.distributedlog.util.DLUtils;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.common.util.PermitLimiter;
import org.apache.distributedlog.util.SimplePermitLimiter;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -86,7 +88,7 @@
}
/**
- * Dynamic DistributedLog Configuration used for the namespace
+ * Dynamic DistributedLog Configuration used for the namespace.
*
* @param dynConf dynamic distributedlog configuration
* @return namespace builder
@@ -110,7 +112,7 @@
}
/**
- * Stats Logger used for stats collection
+ * Stats Logger used for stats collection.
*
* @param statsLogger
* stats logger
@@ -146,7 +148,7 @@
}
/**
- * Client Id used for accessing the namespace
+ * Client Id used for accessing the namespace.
*
* @param clientId
* client id used for accessing the namespace
@@ -175,8 +177,8 @@
StatsLogger perLogStatsLogger,
DistributedLogConfiguration conf) {
StatsLogger normalizedPerLogStatsLogger = perLogStatsLogger;
- if (perLogStatsLogger == NullStatsLogger.INSTANCE &&
- conf.getEnablePerStreamStat()) {
+ if (perLogStatsLogger == NullStatsLogger.INSTANCE
+ && conf.getEnablePerStreamStat()) {
normalizedPerLogStatsLogger = statsLogger.scope("stream");
}
return normalizedPerLogStatsLogger;
@@ -193,8 +195,8 @@
public Namespace build()
throws IllegalArgumentException, NullPointerException, IOException {
// Check arguments
- Preconditions.checkNotNull(_conf, "No DistributedLog Configuration.");
- Preconditions.checkNotNull(_uri, "No DistributedLog URI");
+ checkNotNull(_conf, "No DistributedLog Configuration.");
+ checkNotNull(_uri, "No DistributedLog URI");
// validate the configuration
_conf.validate();
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionStateStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionStateStore.java
index bf4a8d3..3eca2fd 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionStateStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionStateStore.java
@@ -21,19 +21,22 @@
import java.util.concurrent.CompletableFuture;
import org.apache.distributedlog.DLSN;
+/**
+ * Store to manage subscription states.
+ */
public interface SubscriptionStateStore extends Closeable {
/**
- * Get the last committed position stored for this subscription
+ * Get the last committed position stored for this subscription.
*
* @return future represents the last commit position
*/
- public CompletableFuture<DLSN> getLastCommitPosition();
+ CompletableFuture<DLSN> getLastCommitPosition();
/**
- * Advances the position associated with the subscriber
+ * Advances the position associated with the subscriber.
*
* @param newPosition - new commit position
* @return future represents the advance result
*/
- public CompletableFuture<Void> advanceCommitPosition(DLSN newPosition);
+ CompletableFuture<Void> advanceCommitPosition(DLSN newPosition);
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionsStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionsStore.java
index b6a0ed1..d956567 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionsStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/SubscriptionsStore.java
@@ -23,7 +23,7 @@
import org.apache.distributedlog.DLSN;
/**
- * Store to manage subscriptions
+ * Store to manage subscriptions.
*/
public interface SubscriptionsStore extends Closeable {
@@ -34,14 +34,14 @@
* subscriber id
* @return future representing last committed position.
*/
- public CompletableFuture<DLSN> getLastCommitPosition(String subscriberId);
+ CompletableFuture<DLSN> getLastCommitPosition(String subscriberId);
/**
* Get the last committed positions for all subscribers.
*
* @return future representing last committed positions for all subscribers.
*/
- public CompletableFuture<Map<String, DLSN>> getLastCommitPositions();
+ CompletableFuture<Map<String, DLSN>> getLastCommitPositions();
/**
* Advance the last committed position for <i>subscriberId</i>.
@@ -52,7 +52,7 @@
* new committed position.
* @return future representing advancing result.
*/
- public CompletableFuture<Void> advanceCommitPosition(String subscriberId, DLSN newPosition);
+ CompletableFuture<Void> advanceCommitPosition(String subscriberId, DLSN newPosition);
/**
* Delete the subscriber <i>subscriberId</i> permanently. Once the subscriber is deleted, all the
@@ -62,6 +62,6 @@
* return true only if there's such subscriber and we removed it successfully.
* return false if there's no such subscriber, or we failed to remove.
*/
- public CompletableFuture<Boolean> deleteSubscriber(String subscriberId);
+ CompletableFuture<Boolean> deleteSubscriber(String subscriberId);
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/package-info.java
new file mode 100644
index 0000000..277f064
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/api/subscription/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * subscription specific package.
+ *
+ */
+package org.apache.distributedlog.api.subscription;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
index 34011b5..192e6e1 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
@@ -17,41 +17,11 @@
*/
package org.apache.distributedlog.auditor;
-import com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.BookKeeperClient;
-import org.apache.distributedlog.BookKeeperClientBuilder;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.api.namespace.Namespace;
-import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.ZooKeeperClientBuilder;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.util.DLUtils;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.BookKeeperAccessor;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.meta.LedgerManager;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import static com.google.common.base.Charsets.UTF_8;
+import static com.google.common.base.Preconditions.checkArgument;
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
@@ -61,6 +31,7 @@
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
@@ -69,8 +40,44 @@
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAccessor;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.commons.lang3.tuple.Pair;
-import static com.google.common.base.Charsets.UTF_8;
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.BookKeeperClientBuilder;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.util.DLUtils;
+
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+
/**
* DL Auditor will audit DL namespace, e.g. find leaked ledger, report disk usage by streams.
@@ -123,7 +130,7 @@
public Pair<Set<Long>, Set<Long>> collectLedgers(List<URI> uris, List<List<String>> allocationPaths)
throws IOException {
- Preconditions.checkArgument(uris.size() > 0, "No uri provided to audit");
+ checkArgument(uris.size() > 0, "No uri provided to audit");
String zkServers = validateAndGetZKServers(uris);
RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
@@ -212,7 +219,7 @@
throw new DLInterruptedException("Interrupted on collecting ledgers : ", e);
} catch (ExecutionException e) {
if (e.getCause() instanceof IOException) {
- throw (IOException)(e.getCause());
+ throw (IOException) (e.getCause());
} else {
throw new IOException("Failed to collect ledgers : ", e.getCause());
}
@@ -316,7 +323,8 @@
try {
collectLedgersFromPool(poolPath);
} catch (InterruptedException e) {
- throw new DLInterruptedException("Interrupted on collecting ledgers from allocation pool " + poolPath, e);
+ throw new DLInterruptedException("Interrupted on collecting"
+ + " ledgers from allocation pool " + poolPath, e);
} catch (KeeperException e) {
throw new ZKException("Failed to collect ledgers from allocation pool " + poolPath, e.code());
}
@@ -441,7 +449,8 @@
List<LogSegmentMetadata> segments = dlm.getLogSegments();
for (LogSegmentMetadata segment : segments) {
try {
- LedgerHandle lh = getBookKeeperClient(namespace).get().openLedgerNoRecovery(segment.getLogSegmentId(),
+ LedgerHandle lh =
+ getBookKeeperClient(namespace).get().openLedgerNoRecovery(segment.getLogSegmentId(),
BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
totalBytes += lh.getLength();
lh.close();
@@ -517,7 +526,8 @@
executorService.submit(new Runnable() {
@Override
public void run() {
- bk.asyncOpenLedgerNoRecovery(lid, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8),
+ bk.asyncOpenLedgerNoRecovery(lid,
+ BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8),
new org.apache.bookkeeper.client.AsyncCallback.OpenCallback() {
@Override
public void openComplete(int rc, LedgerHandle lh, Object ctx) {
@@ -561,7 +571,7 @@
throw new DLInterruptedException("Interrupted on calculating ledger space : ", e);
} catch (ExecutionException e) {
if (e.getCause() instanceof IOException) {
- throw (IOException)(e.getCause());
+ throw (IOException) (e.getCause());
} else {
throw new IOException("Failed to calculate ledger space : ", e.getCause());
}
@@ -573,8 +583,8 @@
// no-op
}
- static interface Action<T> {
- void execute(T item) throws IOException ;
+ interface Action<T> {
+ void execute(T item) throws IOException;
}
static <T> void executeAction(final LinkedBlockingQueue<T> queue,
@@ -587,7 +597,7 @@
ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
try {
- for (int i = 0 ; i < numThreads; i++) {
+ for (int i = 0; i < numThreads; i++) {
executorService.submit(new Runnable() {
@Override
public void run() {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/package-info.java
new file mode 100644
index 0000000..abb223a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * DistributedLog Auditor.
+ *
+ */
+package org.apache.distributedlog.auditor;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java
index cd3f0ab..36d6cbb 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java
@@ -16,12 +16,15 @@
* limitations under the License.
*/
package org.apache.distributedlog.bk;
-
-import org.apache.distributedlog.util.Allocator;
+import java.io.IOException;
import org.apache.bookkeeper.client.LedgerHandle;
-import java.io.IOException;
+import org.apache.distributedlog.util.Allocator;
+
+/**
+ * Allocator to allocate ledger.
+ */
public interface LedgerAllocator extends Allocator<LedgerHandle, Object> {
/**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
index ee33dc3..f888f6f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
@@ -16,14 +16,15 @@
* limitations under the License.
*/
package org.apache.distributedlog.bk;
-
+import java.io.IOException;
import java.util.concurrent.CompletableFuture;
+
+import org.apache.bookkeeper.client.LedgerHandle;
+
import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.Transaction;
import org.apache.distributedlog.util.Transaction.OpListener;
-import org.apache.bookkeeper.client.LedgerHandle;
-import java.io.IOException;
/**
* Delegator of the underlying allocator. If it owns the allocator, it takes
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
index 19c5546..d877812 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
@@ -19,19 +19,34 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.BookKeeperClient;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.Transaction;
-import org.apache.distributedlog.util.Utils;
+
+import java.util.concurrent.CountDownLatch;
+
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.meta.ZkVersion;
import org.apache.bookkeeper.util.ZkUtils;
import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+
+import org.apache.distributedlog.util.Transaction;
+import org.apache.distributedlog.util.Utils;
+
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -39,21 +54,14 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * LedgerAllocator impl.
+ */
public class LedgerAllocatorPool implements LedgerAllocator {
- static final Logger logger = LoggerFactory.getLogger(LedgerAllocatorPool.class);
+ private static final Logger logger = LoggerFactory.getLogger(LedgerAllocatorPool.class);
private final DistributedLogConfiguration conf;
private final QuorumConfigProvider quorumConfigProvider;
@@ -233,7 +241,7 @@
}
/**
- * Rescue a ledger allocator from an ERROR state
+ * Rescue a ledger allocator from an ERROR state.
* @param ledgerAllocator
* ledger allocator to rescue
*/
@@ -280,9 +288,11 @@
synchronized (LedgerAllocatorPool.this) {
rescueMap.remove(ledgerAllocator.allocatePath);
}
- throw new DLInterruptedException("Interrupted on rescuing ledger allocator " + ledgerAllocator.allocatePath, ie);
+ throw new DLInterruptedException("Interrupted on rescuing ledger allocator "
+ + ledgerAllocator.allocatePath, ie);
} catch (IOException ioe) {
- logger.warn("Failed to rescue ledger allocator {}, retry rescuing it later : ", ledgerAllocator.allocatePath, ioe);
+ logger.warn("Failed to rescue ledger allocator {}, retry rescuing it later : ",
+ ledgerAllocator.allocatePath, ioe);
synchronized (LedgerAllocatorPool.this) {
rescueMap.remove(ledgerAllocator.allocatePath);
}
@@ -295,8 +305,8 @@
SimpleLedgerAllocator allocator;
synchronized (this) {
if (pendingList.isEmpty()) {
- // if no ledger allocator available, we should fail it immediately, which the request will be redirected to other
- // proxies
+ // if no ledger allocator available, we should fail it immediately,
+ // which the request will be redirected to other proxies
throw new IOException("No ledger allocator available under " + poolPath + ".");
} else {
allocator = pendingList.removeFirst();
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorUtils.java
index f6baac4..4184b4c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorUtils.java
@@ -17,13 +17,17 @@
*/
package org.apache.distributedlog.bk;
+import java.io.IOException;
+import java.util.concurrent.ScheduledExecutorService;
import org.apache.distributedlog.BookKeeperClient;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.ZooKeeperClient;
-import java.io.IOException;
-import java.util.concurrent.ScheduledExecutorService;
+
+/**
+ * LedgerAllocator utilities.
+ */
public class LedgerAllocatorUtils {
/**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java
index 07573ff..0cfc0ff 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java
@@ -22,7 +22,7 @@
import org.slf4j.LoggerFactory;
/**
- * Configuration for quorums
+ * Configuration for quorums.
*/
public class QuorumConfig {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfigProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfigProvider.java
index ccf90a9..e48cff3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfigProvider.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfigProvider.java
@@ -18,7 +18,7 @@
package org.apache.distributedlog.bk;
/**
- * Provider to provide quorum config
+ * Provider to provide quorum config.
*/
public interface QuorumConfigProvider {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
index 144b0a6..fbdc3dd 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
@@ -18,33 +18,37 @@
package org.apache.distributedlog.bk;
import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import java.util.function.Function;
-import org.apache.distributedlog.BookKeeperClient;
-import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.util.Transaction;
-import org.apache.distributedlog.util.Transaction.OpListener;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.Utils;
-import org.apache.distributedlog.zk.ZKTransaction;
-import org.apache.distributedlog.zk.ZKVersionedSetOp;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.meta.ZkVersion;
import org.apache.bookkeeper.versioning.Version;
import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.util.DLUtils;
+
+import org.apache.distributedlog.util.Transaction;
+import org.apache.distributedlog.util.Transaction.OpListener;
+import org.apache.distributedlog.util.Utils;
+import org.apache.distributedlog.zk.ZKTransaction;
+import org.apache.distributedlog.zk.ZKVersionedSetOp;
+
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
+
/**
* Allocator to allocate ledgers.
@@ -53,7 +57,7 @@
static final Logger LOG = LoggerFactory.getLogger(SimpleLedgerAllocator.class);
- static enum Phase {
+ enum Phase {
ALLOCATING, ALLOCATED, HANDING_OVER, HANDED_OVER, ERROR
}
@@ -218,7 +222,8 @@
private synchronized void deleteLedgerLeftFromPreviousAllocationIfNecessary() {
if (null != ledgerIdLeftFromPrevAllocation) {
- LOG.info("Deleting allocated-but-unused ledger left from previous allocation {}.", ledgerIdLeftFromPrevAllocation);
+ LOG.info("Deleting allocated-but-unused ledger left from previous allocation {}.",
+ ledgerIdLeftFromPrevAllocation);
deleteLedger(ledgerIdLeftFromPrevAllocation);
ledgerIdLeftFromPrevAllocation = null;
}
@@ -293,8 +298,8 @@
OpListener<LedgerHandle> listenerToNotify;
synchronized (this) {
listenerToNotify = tryObtainListener;
- if (t instanceof KeeperException &&
- ((KeeperException) t).code() == KeeperException.Code.BADVERSION) {
+ if (t instanceof KeeperException
+ && ((KeeperException) t).code() == KeeperException.Code.BADVERSION) {
LOG.info("Set ledger allocator {} to ERROR state after hit bad version : version = {}",
allocatePath, getVersion());
setPhase(Phase.ERROR);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java
index 20ed48b..88a0535 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java
@@ -17,12 +17,13 @@
*/
package org.apache.distributedlog.callback;
-import org.apache.distributedlog.LogSegmentMetadata;
-
import java.util.List;
+import org.apache.distributedlog.LogSegmentMetadata;
+
+
/**
- * Listener on log segments changes for a given stream used by {@link org.apache.distributedlog.BKLogReadHandler}
+ * Listener on log segments changes for a given stream used by {@link org.apache.distributedlog.BKLogReadHandler}.
*/
public interface LogSegmentListener {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java
index 21fd155..ca4f581 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java
@@ -17,9 +17,10 @@
*/
package org.apache.distributedlog.callback;
+import java.util.List;
+
import org.apache.bookkeeper.versioning.Versioned;
-import java.util.List;
/**
* Listener on list of log segments changes for a given stream used by
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java
index 443c1f8..4b75fd4 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java
@@ -21,6 +21,9 @@
import java.util.Iterator;
+/**
+ * Namespace Listener.
+ */
@Beta
public interface NamespaceListener {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java
index c69b7a5..3753014 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.java
@@ -17,12 +17,11 @@
*/
package org.apache.distributedlog.config;
+import static com.google.common.base.Preconditions.checkNotNull;
+
import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
-import org.apache.distributedlog.DistributedLogConfiguration;
-
import java.io.File;
import java.net.MalformedURLException;
import java.util.HashMap;
@@ -32,7 +31,11 @@
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
+
+
+
import org.apache.commons.configuration.ConfigurationException;
+import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
@@ -57,7 +60,8 @@
private final int reloadPeriod;
private final TimeUnit reloadUnit;
- public DynamicConfigurationFactory(ScheduledExecutorService executorService, int reloadPeriod, TimeUnit reloadUnit) {
+ public DynamicConfigurationFactory(ScheduledExecutorService executorService,
+ int reloadPeriod, TimeUnit reloadUnit) {
this.executorService = executorService;
this.reloadPeriod = reloadPeriod;
this.reloadUnit = reloadUnit;
@@ -68,7 +72,7 @@
public synchronized Optional<DynamicDistributedLogConfiguration> getDynamicConfiguration(
String configPath,
ConcurrentBaseConfiguration defaultConf) throws ConfigurationException {
- Preconditions.checkNotNull(configPath);
+ checkNotNull(configPath);
try {
if (!dynamicConfigs.containsKey(configPath)) {
File configFile = new File(configPath);
@@ -89,7 +93,8 @@
}
}
- public synchronized Optional<DynamicDistributedLogConfiguration> getDynamicConfiguration(String configPath) throws ConfigurationException {
+ public synchronized Optional<DynamicDistributedLogConfiguration>
+ getDynamicConfiguration(String configPath) throws ConfigurationException {
return getDynamicConfiguration(configPath, new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
index 9e760c5..88c114c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
@@ -17,11 +17,16 @@
*/
package org.apache.distributedlog.config;
+import static org.apache.distributedlog.DistributedLogConfiguration.*;
+
+
+
import org.apache.distributedlog.DistributedLogConfiguration;
+
import org.apache.distributedlog.bk.QuorumConfig;
import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
-import static org.apache.distributedlog.DistributedLogConfiguration.*;
+
/**
* Whitelist dynamic configuration by adding an accessor to this class.
@@ -42,7 +47,7 @@
}
/**
- * Get retention period in hours
+ * Get retention period in hours.
*
* @return retention period in hours
*/
@@ -193,7 +198,7 @@
}
/**
- * Get output buffer size
+ * Get output buffer size.
*
* @return buffer size
*/
@@ -210,7 +215,7 @@
}
/**
- * Get Periodic Log Flush Frequency in seconds
+ * Get Periodic Log Flush Frequency in seconds.
*
* @return periodic flush frequency
*/
@@ -221,7 +226,7 @@
}
/**
- * Get the number of entries that readahead worker reads as a batch from bookkeeper
+ * Get the number of entries that readahead worker reads as a batch from bookkeeper.
*
* @return the batch size
*/
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/package-info.java
index bc9c512..e107efe 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/config/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * DistributedLog Configuration
+ * DistributedLog Configuration.
*/
package org.apache.distributedlog.config;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java
index 5eab707..c4bfbb6 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java
@@ -21,7 +21,7 @@
import org.apache.zookeeper.KeeperException.Code;
/**
- * TODO: move ZKException to distributedlog-protocol
+ * TODO: move ZKException to distributedlog-protocol.
*/
public class ZKException extends DLException {
@@ -45,9 +45,9 @@
public static boolean isRetryableZKException(ZKException zke) {
KeeperException.Code code = zke.getKeeperExceptionCode();
- return KeeperException.Code.CONNECTIONLOSS == code ||
- KeeperException.Code.OPERATIONTIMEOUT == code ||
- KeeperException.Code.SESSIONEXPIRED == code ||
- KeeperException.Code.SESSIONMOVED == code;
+ return KeeperException.Code.CONNECTIONLOSS == code
+ || KeeperException.Code.OPERATIONTIMEOUT == code
+ || KeeperException.Code.SESSIONEXPIRED == code
+ || KeeperException.Code.SESSIONMOVED == code;
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/package-info.java
new file mode 100644
index 0000000..eed88d1
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * DistributedLog zk Exception.
+ */
+package org.apache.distributedlog.exceptions;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java
index df92b95..86b352f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java
@@ -17,25 +17,28 @@
*/
package org.apache.distributedlog.feature;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.bookkeeper.feature.CacheableFeatureProvider;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.commons.configuration.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
+import org.apache.bookkeeper.feature.CacheableFeatureProvider;
+import org.apache.bookkeeper.feature.Feature;
+
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.distributedlog.DistributedLogConfiguration;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
/**
- * Decider based feature provider
+ * Decider based feature provider.
*/
public abstract class AbstractFeatureProvider<T extends Feature> extends CacheableFeatureProvider<T> {
- protected static final Logger logger = LoggerFactory.getLogger(AbstractFeatureProvider.class);
+ private static final Logger logger = LoggerFactory.getLogger(AbstractFeatureProvider.class);
public static FeatureProvider getFeatureProvider(String rootScope,
DistributedLogConfiguration conf,
@@ -68,7 +71,8 @@
if (targetException instanceof IOException) {
throw (IOException) targetException;
} else {
- throw new IOException("Encountered invocation target exception while instantiating feature provider : ", e);
+ throw new IOException("Encountered invocation target"
+ + " exception while instantiating feature provider : ", e);
}
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
index f51302e..7475bf7 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
@@ -17,18 +17,19 @@
*/
package org.apache.distributedlog.feature;
-import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+import java.util.concurrent.ConcurrentMap;
import org.apache.bookkeeper.feature.CacheableFeatureProvider;
import org.apache.bookkeeper.feature.Feature;
import org.apache.bookkeeper.feature.FeatureProvider;
import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.concurrent.ConcurrentMap;
/**
- * Feature Provider that load features from configuration
+ * Feature Provider that load features from configuration.
*/
class ConfigurationFeatureProvider extends CacheableFeatureProvider {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/CoreFeatureKeys.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/CoreFeatureKeys.java
index bc531ef..ad0c07e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/CoreFeatureKeys.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/CoreFeatureKeys.java
@@ -18,7 +18,7 @@
package org.apache.distributedlog.feature;
/**
- * List of feature keys used by distributedlog core
+ * List of feature keys used by distributedlog core.
*/
public enum CoreFeatureKeys {
// @Deprecated: bkc features are managed by bookkeeper prefixed with a scope
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DefaultFeatureProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DefaultFeatureProvider.java
index 6c30ab3..6aa3888 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DefaultFeatureProvider.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DefaultFeatureProvider.java
@@ -17,12 +17,13 @@
*/
package org.apache.distributedlog.feature;
-import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.bookkeeper.feature.Feature;
import org.apache.bookkeeper.feature.FeatureProvider;
import org.apache.bookkeeper.feature.SettableFeature;
import org.apache.bookkeeper.feature.SettableFeatureProvider;
import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.DistributedLogConfiguration;
+
/**
* Default feature provider which disable all features by default.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
index 201ed8a..bec6f58 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
@@ -17,21 +17,9 @@
*/
package org.apache.distributedlog.feature;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkNotNull;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
-import org.apache.distributedlog.common.config.ConfigurationListener;
-import org.apache.distributedlog.common.config.ConfigurationSubscription;
-import org.apache.distributedlog.common.config.FileConfigurationBuilder;
-import org.apache.distributedlog.common.config.PropertiesConfigurationBuilder;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.commons.configuration.ConfigurationException;
-
import java.io.File;
import java.io.IOException;
import java.util.List;
@@ -41,12 +29,28 @@
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+import org.apache.distributedlog.common.config.ConfigurationListener;
+import org.apache.distributedlog.common.config.ConfigurationSubscription;
+import org.apache.distributedlog.common.config.FileConfigurationBuilder;
+import org.apache.distributedlog.common.config.PropertiesConfigurationBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
/**
* Feature Provider based dynamic configuration.
*/
public class DynamicConfigurationFeatureProvider extends AbstractFeatureProvider
implements ConfigurationListener {
+ private static final Logger logger = LoggerFactory.getLogger(DynamicConfigurationFeatureProvider.class);
+
private final ConcurrentBaseConfiguration featuresConf;
private ConfigurationSubscription featuresConfSubscription;
@@ -76,7 +80,7 @@
List<FileConfigurationBuilder> fileConfigBuilders =
Lists.newArrayListWithExpectedSize(2);
String baseConfigPath = conf.getFileFeatureProviderBaseConfigPath();
- Preconditions.checkNotNull(baseConfigPath);
+ checkNotNull(baseConfigPath);
File baseConfigFile = new File(baseConfigPath);
FileConfigurationBuilder baseProperties =
new PropertiesConfigurationBuilder(baseConfigFile.toURI().toURL());
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/package-info.java
index 4abe31b..c3c242d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * Feature & FeatureProvider for distributedlog
+ * Feature & FeatureProvider for distributedlog.
*/
package org.apache.distributedlog.feature;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetLastTxIdFunction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetLastTxIdFunction.java
index 00703e3..55c2c69 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetLastTxIdFunction.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetLastTxIdFunction.java
@@ -23,7 +23,7 @@
import org.apache.distributedlog.LogSegmentMetadata;
/**
- * Retrieve the last tx id from list of log segments
+ * Retrieve the last tx id from list of log segments.
*/
public class GetLastTxIdFunction implements Function<List<LogSegmentMetadata>, Long> {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/function/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/function/package-info.java
index ed792d9..16c2d90 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/function/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/function/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * Common Functions for DistributedLog
+ * Common Functions for DistributedLog.
*/
package org.apache.distributedlog.function;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
index 36ff437..30c42cb 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
@@ -17,6 +17,9 @@
*/
package org.apache.distributedlog.impl;
+import static org.apache.distributedlog.util.DLUtils.isReservedStreamName;
+import static org.apache.distributedlog.util.DLUtils.validateAndNormalizeName;
+
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -24,42 +27,55 @@
import io.netty.channel.epoll.EpollEventLoopGroup;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.util.HashedWheelTimer;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
import org.apache.commons.lang.SystemUtils;
+
import org.apache.distributedlog.BookKeeperClient;
import org.apache.distributedlog.BookKeeperClientBuilder;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.api.MetadataAccessor;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.ZooKeeperClientBuilder;
+
import org.apache.distributedlog.acl.AccessControlManager;
import org.apache.distributedlog.acl.DefaultAccessControlManager;
-import org.apache.distributedlog.impl.acl.ZKAccessControlManager;
+import org.apache.distributedlog.api.MetadataAccessor;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
import org.apache.distributedlog.bk.LedgerAllocator;
import org.apache.distributedlog.bk.LedgerAllocatorUtils;
+
+
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.exceptions.AlreadyClosedException;
import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.impl.acl.ZKAccessControlManager;
import org.apache.distributedlog.impl.federated.FederatedZKLogMetadataStore;
import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
import org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore;
import org.apache.distributedlog.impl.subscription.ZKSubscriptionsStore;
import org.apache.distributedlog.injector.AsyncFailureInjector;
import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
import org.apache.distributedlog.metadata.LogMetadataForReader;
import org.apache.distributedlog.metadata.LogMetadataStore;
import org.apache.distributedlog.metadata.LogStreamMetadataStore;
import org.apache.distributedlog.namespace.NamespaceDriver;
import org.apache.distributedlog.namespace.NamespaceDriverManager;
-import org.apache.distributedlog.api.subscription.SubscriptionsStore;
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Utils;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
+
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.common.PathUtils;
@@ -67,19 +83,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import static org.apache.distributedlog.util.DLUtils.isReservedStreamName;
-import static org.apache.distributedlog.util.DLUtils.validateAndNormalizeName;
+
/**
- * Manager for ZooKeeper/BookKeeper based namespace
+ * Manager for ZooKeeper/BookKeeper based namespace.
*/
public class BKNamespaceDriver implements NamespaceDriver {
@@ -327,11 +335,12 @@
}
@VisibleForTesting
- public static String validateAndGetFullLedgerAllocatorPoolPath(DistributedLogConfiguration conf, URI uri) throws IOException {
+ public static String
+ validateAndGetFullLedgerAllocatorPoolPath(DistributedLogConfiguration conf, URI uri) throws IOException {
String poolPath = conf.getLedgerAllocatorPoolPath();
LOG.info("PoolPath is {}", poolPath);
if (null == poolPath || !poolPath.startsWith(".") || poolPath.endsWith("/")) {
- LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool : {}", poolPath);
+ LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool: {}", poolPath);
throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
}
String poolName = conf.getLedgerAllocatorPoolName();
@@ -343,7 +352,7 @@
try {
PathUtils.validatePath(rootPath);
} catch (IllegalArgumentException iae) {
- LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool : {}", poolPath);
+ LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool: {}", poolPath);
throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
}
return rootPath;
@@ -363,7 +372,8 @@
if (null != allocator) {
allocator.start();
}
- LOG.info("Created ledger allocator pool under {} with size {}.", allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize());
+ LOG.info("Created ledger allocator pool under {} with size {}.",
+ allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize());
} else {
allocator = null;
}
@@ -539,7 +549,7 @@
return result;
}
List<String> children = zk.getChildren(namespaceRootPath, false);
- for(String child: children) {
+ for (String child: children) {
if (isReservedStreamName(child)) {
continue;
}
@@ -584,10 +594,10 @@
.retryPolicy(retryPolicy)
.statsLogger(statsLogger)
.zkAclId(conf.getZkAclId());
- LOG.info("Created shared zooKeeper client builder {}: zkServers = {}, numRetries = {}, sessionTimeout = {}, retryBackoff = {},"
- + " maxRetryBackoff = {}, zkAclId = {}.", new Object[] { zkcName, zkServers, conf.getZKNumRetries(),
- conf.getZKSessionTimeoutMilliseconds(), conf.getZKRetryBackoffStartMillis(),
- conf.getZKRetryBackoffMaxMillis(), conf.getZkAclId() });
+ LOG.info("Created shared zooKeeper client builder {}: zkServers = {}, numRetries = {}, sessionTimeout = {},"
+ + " retryBackoff = {}, maxRetryBackoff = {}, zkAclId = {}.", new Object[] {
+ zkcName, zkServers, conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(),
+ conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(), conf.getZkAclId() });
return builder;
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
index 7069cbb..43fed26 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
@@ -17,6 +17,8 @@
*/
package org.apache.distributedlog.impl;
+import static org.apache.distributedlog.util.DLUtils.*;
+
import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import java.net.URI;
@@ -26,19 +28,18 @@
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.ZKException;
import org.apache.distributedlog.metadata.LogMetadataStore;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.Stat;
-import static org.apache.distributedlog.util.DLUtils.*;
/**
- * ZooKeeper based log metadata store
+ * ZooKeeper based log metadata store.
*/
public class ZKLogMetadataStore implements LogMetadataStore {
@@ -80,7 +81,8 @@
if (KeeperException.Code.OK.intValue() == syncRc) {
zk.getChildren(nsRootPath, false, new AsyncCallback.Children2Callback() {
@Override
- public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+ public void processResult(int rc, String path, Object ctx,
+ List<String> children, Stat stat) {
if (KeeperException.Code.OK.intValue() == rc) {
List<String> results = Lists.newArrayListWithExpectedSize(children.size());
for (String child : children) {
@@ -93,8 +95,8 @@
List<String> streams = Lists.newLinkedList();
promise.complete(streams.iterator());
} else {
- promise.completeExceptionally(new ZKException("Error reading namespace " + nsRootPath,
- KeeperException.Code.get(rc)));
+ promise.completeExceptionally(new ZKException("Error reading namespace "
+ + nsRootPath, KeeperException.Code.get(rc)));
}
}
}, null);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java
index ca35c1c..4e64161 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java
@@ -17,14 +17,15 @@
*/
package org.apache.distributedlog.impl;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
import org.apache.distributedlog.DistributedLogConstants;
import org.apache.distributedlog.logsegment.LogSegmentFilter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
+
/**
* Filters based on current zookeeper log segments.
@@ -57,7 +58,7 @@
lastCompletedLogSegmentName = s;
}
} else if (6 == parts.length) {
- // name: logrecs_<start_tx_id>_<end_tx_id>_<logsegment_sequence_number>_<ledger_id>_<region_id>
+ //name: logrecs_<start_tx_id>_<end_tx_id>_<logsegment_seq_number>_<ledger_id>_<region_id>
long logSegmentSequenceNumber = Long.parseLong(parts[3]);
if (logSegmentSequenceNumber > lastLogSegmentSequenceNumber) {
lastLogSegmentSequenceNumber = logSegmentSequenceNumber;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
index f747045..d11fa02 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
@@ -17,20 +17,36 @@
*/
package org.apache.distributedlog.impl;
+import static com.google.common.base.Charsets.UTF_8;
+
import com.google.common.collect.ImmutableList;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.LogSegmentMetadata;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
import org.apache.distributedlog.exceptions.LogNotFoundException;
import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
import org.apache.distributedlog.metadata.LogMetadata;
import org.apache.distributedlog.metadata.LogMetadataForWriter;
-import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Transaction;
import org.apache.distributedlog.util.Transaction.OpListener;
@@ -39,9 +55,7 @@
import org.apache.distributedlog.zk.ZKOp;
import org.apache.distributedlog.zk.ZKTransaction;
import org.apache.distributedlog.zk.ZKVersionedSetOp;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
+
import org.apache.zookeeper.AsyncCallback.Children2Callback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -52,18 +66,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import static com.google.common.base.Charsets.UTF_8;
/**
* ZooKeeper based log segment metadata store.
@@ -137,8 +140,9 @@
}
synchronized void onSegmentsUpdated(Versioned<List<String>> logSegments) {
- if (lastNotifiedLogSegments.getVersion() == Version.NEW ||
- lastNotifiedLogSegments.getVersion().compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
+ if (lastNotifiedLogSegments.getVersion() == Version.NEW
+ || lastNotifiedLogSegments.getVersion()
+ .compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
lastNotifiedLogSegments = logSegments;
listener.onSegmentsUpdated(logSegments);
}
@@ -320,7 +324,7 @@
// reads
/**
- * Process the watched events for registered listeners
+ * Process the watched events for registered listeners.
*/
@Override
public void process(WatchedEvent event) {
@@ -421,7 +425,8 @@
closeLock.readLock().unlock();
}
}
- CompletableFuture<Versioned<List<String>>> getLogSegmentNamesResult = zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
+ CompletableFuture<Versioned<List<String>>> getLogSegmentNamesResult =
+ zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
if (null != listener) {
getLogSegmentNamesResult.whenComplete(new ReadLogSegmentsTask(logSegmentsPath, this));
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
index b3fe456..ad86b57 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
@@ -17,30 +17,35 @@
*/
package org.apache.distributedlog.impl;
+import static org.apache.distributedlog.impl.BKNamespaceDriver.getZKServersFromDLUri;
+import com.google.common.annotations.VisibleForTesting;
+
import java.io.IOException;
import java.net.URI;
-import com.google.common.annotations.VisibleForTesting;
import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.api.MetadataAccessor;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.ZooKeeperClientBuilder;
-import org.apache.distributedlog.exceptions.AlreadyClosedException;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.Utils;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.api.MetadataAccessor;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.util.Utils;
+
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.distributedlog.impl.BKNamespaceDriver.getZKServersFromDLUri;
+/**
+ * access to ZKMetadata.
+ */
public class ZKMetadataAccessor implements MetadataAccessor {
static final Logger LOG = LoggerFactory.getLogger(ZKMetadataAccessor.class);
protected final String name;
@@ -131,7 +136,7 @@
}
/**
- * Get the name of the stream managed by this log manager
+ * Get the name of the stream managed by this log manager.
*
* @return streamName
*/
@@ -142,7 +147,7 @@
/**
* Creates or update the metadata stored at the node associated with the
- * name and URI
+ * name and URI.
* @param metadata opaque metadata to be stored for the node
* @throws IOException
*/
@@ -184,8 +189,8 @@
}
/**
- * Retrieve the metadata stored at the node
- * @return byte array containing the metadata
+ * Retrieve the metadata stored at the node.
+ * @return byte array containing the metadata.
* @throws IOException
*/
@Override
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java
index 621cb2d..f14c5bd 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java
@@ -17,6 +17,14 @@
*/
package org.apache.distributedlog.impl;
+import static org.apache.distributedlog.util.DLUtils.*;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.callback.NamespaceListener;
@@ -30,22 +38,15 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import static org.apache.distributedlog.util.DLUtils.*;
/**
- * Watcher on watching a given namespace
+ * Watcher on watching a given namespace.
*/
public class ZKNamespaceWatcher extends NamespaceWatcher
implements Runnable, Watcher, AsyncCallback.Children2Callback {
- static final Logger logger = LoggerFactory.getLogger(ZKNamespaceWatcher.class);
+ private static final Logger logger = LoggerFactory.getLogger(ZKNamespaceWatcher.class);
private final DistributedLogConfiguration conf;
private final URI uri;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
index e4a175c..ffc6e57 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
@@ -16,10 +16,15 @@
* limitations under the License.
*/
package org.apache.distributedlog.impl.acl;
+import static com.google.common.base.Charsets.UTF_8;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
import java.util.concurrent.CompletableFuture;
+
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.thrift.AccessControlEntry;
import org.apache.thrift.TException;
@@ -32,17 +37,17 @@
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-
-import static com.google.common.base.Charsets.UTF_8;
-
+/**
+ * ZooKeeper Based AccessControl.
+ */
public class ZKAccessControl {
private static final int BUFFER_SIZE = 4096;
public static final AccessControlEntry DEFAULT_ACCESS_CONTROL_ENTRY = new AccessControlEntry();
-
+ /**
+ * AccessControl Exception.
+ */
public static class CorruptedAccessControlException extends IOException {
private static final long serialVersionUID = 5391285182476211603L;
@@ -77,8 +82,8 @@
return false;
}
ZKAccessControl other = (ZKAccessControl) obj;
- return Objects.equal(zkPath, other.zkPath) &&
- Objects.equal(accessControlEntry, other.accessControlEntry);
+ return Objects.equal(zkPath, other.zkPath)
+ && Objects.equal(accessControlEntry, other.accessControlEntry);
}
@Override
@@ -148,7 +153,8 @@
return promise;
}
- public static CompletableFuture<ZKAccessControl> read(final ZooKeeperClient zkc, final String zkPath, Watcher watcher) {
+ public static CompletableFuture<ZKAccessControl>
+ read(final ZooKeeperClient zkc, final String zkPath, Watcher watcher) {
final CompletableFuture<ZKAccessControl> promise = new CompletableFuture<ZKAccessControl>();
try {
@@ -182,8 +188,8 @@
zkc.get().delete(zkPath, -1, new AsyncCallback.VoidCallback() {
@Override
public void processResult(int rc, String path, Object ctx) {
- if (KeeperException.Code.OK.intValue() == rc ||
- KeeperException.Code.NONODE.intValue() == rc) {
+ if (KeeperException.Code.OK.intValue() == rc
+ || KeeperException.Code.NONODE.intValue() == rc) {
promise.complete(null);
} else {
promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
index 3dbde6a..83e355f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
@@ -18,15 +18,24 @@
package org.apache.distributedlog.impl.acl;
import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.util.ZkUtils;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.acl.AccessControlManager;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.thrift.AccessControlEntry;
-import org.apache.bookkeeper.util.ZkUtils;
import org.apache.distributedlog.common.concurrent.FutureEventListener;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.thrift.AccessControlEntry;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -37,18 +46,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
+
/**
- * ZooKeeper Based {@link org.apache.distributedlog.acl.AccessControlManager}
+ * ZooKeeper Based {@link org.apache.distributedlog.acl.AccessControlManager}.
*/
public class ZKAccessControlManager implements AccessControlManager, Watcher {
@@ -78,13 +79,16 @@
FutureUtils.result(fetchDefaultAccessControlEntry());
} catch (Throwable t) {
if (t instanceof InterruptedException) {
- throw new DLInterruptedException("Interrupted on getting default access control entry for " + zkRootPath, t);
+ throw new DLInterruptedException("Interrupted on getting default access control entry for "
+ + zkRootPath, t);
} else if (t instanceof KeeperException) {
- throw new IOException("Encountered zookeeper exception on getting default access control entry for " + zkRootPath, t);
+ throw new IOException("Encountered zookeeper exception on getting default access control entry for "
+ + zkRootPath, t);
} else if (t instanceof IOException) {
throw (IOException) t;
} else {
- throw new IOException("Encountered unknown exception on getting access control entries for " + zkRootPath, t);
+ throw new IOException("Encountered unknown exception on getting access control entries for "
+ + zkRootPath, t);
}
}
@@ -94,11 +98,13 @@
if (t instanceof InterruptedException) {
throw new DLInterruptedException("Interrupted on getting access control entries for " + zkRootPath, t);
} else if (t instanceof KeeperException) {
- throw new IOException("Encountered zookeeper exception on getting access control entries for " + zkRootPath, t);
+ throw new IOException("Encountered zookeeper exception on getting access control entries for "
+ + zkRootPath, t);
} else if (t instanceof IOException) {
throw (IOException) t;
} else {
- throw new IOException("Encountered unknown exception on getting access control entries for " + zkRootPath, t);
+ throw new IOException("Encountered unknown exception on getting access control entries for "
+ + zkRootPath, t);
}
}
}
@@ -161,7 +167,8 @@
for (String s : streamsRemoved) {
ZKAccessControl accessControl = streamEntries.remove(s);
if (null != accessControl) {
- logger.info("Removed Access Control Entry for stream {} : {}", s, accessControl.getAccessControlEntry());
+ logger.info("Removed Access Control Entry for stream {} : {}",
+ s, accessControl.getAccessControlEntry());
}
}
if (streamsReceived.isEmpty()) {
@@ -178,7 +185,8 @@
@Override
public void onSuccess(ZKAccessControl accessControl) {
streamEntries.put(streamName, accessControl);
- logger.info("Added overrided access control for stream {} : {}", streamName, accessControl.getAccessControlEntry());
+ logger.info("Added overrided access control for stream {} : {}",
+ streamName, accessControl.getAccessControlEntry());
complete();
}
@@ -187,7 +195,7 @@
if (cause instanceof KeeperException.NoNodeException) {
streamEntries.remove(streamName);
} else if (cause instanceof ZKAccessControl.CorruptedAccessControlException) {
- logger.warn("Access control is corrupted for stream {} @ {}, skipped it ...",
+ logger.warn("Access control is corrupted for stream {} @ {},skipped it ...",
new Object[] { streamName, zkRootPath, cause });
streamEntries.remove(streamName);
} else {
@@ -288,8 +296,8 @@
return;
}
- logger.warn("Encountered an error on refetching default access control entry, retrying in {} ms : ",
- ZK_RETRY_BACKOFF_MS, cause);
+ logger.warn("Encountered an error on refetching default access control entry,"
+ + " retrying in {} ms : ", ZK_RETRY_BACKOFF_MS, cause);
refetchDefaultAccessControlEntry(ZK_RETRY_BACKOFF_MS);
}
});
@@ -338,8 +346,8 @@
@Override
public void onFailure(Throwable cause) {
- logger.warn("Encountered an error on fetching all access control entries, retrying in {} ms : ",
- ZK_RETRY_BACKOFF_MS, cause);
+ logger.warn("Encountered an error on fetching all"
+ + " access control entries, retrying in {} ms : ", ZK_RETRY_BACKOFF_MS, cause);
refetchAccessControlEntries(ZK_RETRY_BACKOFF_MS);
}
});
@@ -347,8 +355,8 @@
@Override
public void onFailure(Throwable cause) {
- logger.warn("Encountered an error on refetching all access control entries, retrying in {} ms : ",
- ZK_RETRY_BACKOFF_MS, cause);
+ logger.warn("Encountered an error on refetching all"
+ + " access control entries, retrying in {} ms : ", ZK_RETRY_BACKOFF_MS, cause);
refetchAllAccessControlEntries(ZK_RETRY_BACKOFF_MS);
}
});
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/package-info.java
new file mode 100644
index 0000000..18f6b40
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Access Control impl for distributedlog streams.
+ */
+package org.apache.distributedlog.impl.acl;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
index 17515c3..5187dfc 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
@@ -17,24 +17,43 @@
*/
package org.apache.distributedlog.impl.federated;
+import static com.google.common.base.Charsets.UTF_8;
+
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
import com.google.common.base.Optional;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
import java.util.concurrent.CompletableFuture;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.LogExistsException;
import org.apache.distributedlog.exceptions.UnexpectedException;
import org.apache.distributedlog.exceptions.ZKException;
import org.apache.distributedlog.impl.ZKNamespaceWatcher;
import org.apache.distributedlog.metadata.LogMetadataStore;
import org.apache.distributedlog.namespace.NamespaceWatcher;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
+
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Utils;
import org.apache.zookeeper.AsyncCallback;
@@ -49,42 +68,27 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import static com.google.common.base.Charsets.UTF_8;
+
/**
* A Federated ZooKeeper Based Log Metadata Store.
- *
- * To Upgrade a simple ZKLogMetadataStore to FederatedZKLogMetadataStore, following steps should be taken in sequence:
+ *To Upgrade a simple ZKLogMetadataStore to FederatedZKLogMetadataStore, following steps should be taken in sequence:
* a) deploy the new code with disabling createStreamsIfNotExists in all writer.
* b) once all proxies disable the flag, update namespace binding to enable federated namespace.
* c) restart writers to take federated namespace in place.
*
- * NOTE: current federated namespace isn't optimized for deletion/creation. so don't use it in the workloads
- * that have lots of creations or deletions.
+ * <p>NOTE: current federated namespace isn't optimized for deletion/creation. so don't use it in the workloads
+ * that have lots of creations or deletions.</p>
*/
public class FederatedZKLogMetadataStore
extends NamespaceWatcher
implements LogMetadataStore, Watcher, Runnable, FutureEventListener<Set<URI>> {
- static final Logger logger = LoggerFactory.getLogger(FederatedZKLogMetadataStore.class);
+ private static final Logger logger = LoggerFactory.getLogger(FederatedZKLogMetadataStore.class);
- private final static String ZNODE_SUB_NAMESPACES = ".subnamespaces";
- private final static String SUB_NAMESPACE_PREFIX = "NS_";
+ private static final String ZNODE_SUB_NAMESPACES = ".subnamespaces";
+ private static final String SUB_NAMESPACE_PREFIX = "NS_";
/**
* Create the federated namespace.
@@ -148,8 +152,8 @@
for (String logName : newLogs) {
URI oldURI = log2Locations.putIfAbsent(logName, uri);
if (null != oldURI && !Objects.equal(uri, oldURI)) {
- logger.error("Log {} is found duplicated in multiple locations : old location = {}," +
- " new location = {}", new Object[] { logName, oldURI, uri });
+ logger.error("Log {} is found duplicated in multiple locations : old location = {},"
+ + " new location = {}", new Object[] { logName, oldURI, uri });
duplicatedLogFound.set(true);
}
}
@@ -241,7 +245,8 @@
@Override
public void onSuccess(T value) {
if (duplicatedLogFound.get()) {
- postCheckedPromise.completeExceptionally(new UnexpectedException("Duplicate log found under " + namespace));
+ postCheckedPromise.completeExceptionally(
+ new UnexpectedException("Duplicate log found under " + namespace));
} else {
postCheckedPromise.complete(value);
}
@@ -370,8 +375,8 @@
@Override
public void process(WatchedEvent watchedEvent) {
- if (Event.EventType.None == watchedEvent.getType() &&
- Event.KeeperState.Expired == watchedEvent.getState()) {
+ if (Event.EventType.None == watchedEvent.getType()
+ && Event.KeeperState.Expired == watchedEvent.getState()) {
scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
return;
}
@@ -385,7 +390,7 @@
// Log Related Methods
//
- private <A> CompletableFuture<A> duplicatedLogException(String logName) {
+ private <T> CompletableFuture<T> duplicatedLogException(String logName) {
return FutureUtils.exception(new UnexpectedException("Duplicated log " + logName
+ " found in namespace " + namespace));
}
@@ -405,7 +410,8 @@
@Override
public void onSuccess(Optional<URI> uriOptional) {
if (uriOptional.isPresent()) {
- createPromise.completeExceptionally(new LogExistsException("Log " + logName + " already exists in " + uriOptional.get()));
+ createPromise.completeExceptionally(
+ new LogExistsException("Log " + logName + " already exists in " + uriOptional.get()));
} else {
getCachedSubNamespacesAndCreateLog(logName, createPromise);
}
@@ -523,7 +529,8 @@
} catch (UnexpectedException ue) {
promise.completeExceptionally(ue);
} catch (URISyntaxException e) {
- promise.completeExceptionally(new UnexpectedException("Invalid namespace " + name + " is created."));
+ promise.completeExceptionally(
+ new UnexpectedException("Invalid namespace " + name + " is created."));
}
} else {
promise.completeExceptionally(KeeperException.create(Code.get(rc)));
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/package-info.java
new file mode 100644
index 0000000..22de8a7
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Access Control impl for distributedlog streams.
+ */
+package org.apache.distributedlog.impl.federated;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
index e45c755..2ae27e2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
@@ -20,14 +20,15 @@
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
+import org.apache.bookkeeper.client.LedgerHandle;
+
import org.apache.distributedlog.bk.LedgerAllocator;
import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
import org.apache.distributedlog.util.Allocator;
import org.apache.distributedlog.util.Transaction;
-import org.apache.bookkeeper.client.LedgerHandle;
/**
- * Allocate log segments
+ * Allocate log segments.
*/
class BKLogSegmentAllocator implements Allocator<LogSegmentEntryWriter, Object> {
@@ -56,8 +57,8 @@
}
@Override
- public CompletableFuture<LogSegmentEntryWriter> tryObtain(Transaction<Object> txn,
- final Transaction.OpListener<LogSegmentEntryWriter> listener) {
+ public CompletableFuture<LogSegmentEntryWriter> tryObtain(Transaction<Object> txn, final
+ Transaction.OpListener<LogSegmentEntryWriter> listener) {
return allocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
@Override
public void onCommit(LedgerHandle lh) {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
index b1dbf40..a3837ae 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
@@ -33,18 +33,6 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.Entry;
-import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.exceptions.BKTransmitException;
-import org.apache.distributedlog.exceptions.DLIllegalStateException;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.exceptions.EndOfLogSegmentException;
-import org.apache.distributedlog.exceptions.ReadCancelledException;
-import org.apache.distributedlog.injector.AsyncFailureInjector;
-import org.apache.distributedlog.logsegment.LogSegmentEntryReader;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.bookkeeper.client.AsyncCallback;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
@@ -52,6 +40,19 @@
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.stats.Counter;
import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.Entry;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.EndOfLogSegmentException;
+import org.apache.distributedlog.exceptions.ReadCancelledException;
+import org.apache.distributedlog.injector.AsyncFailureInjector;
+import org.apache.distributedlog.logsegment.LogSegmentEntryReader;
+import org.apache.distributedlog.util.OrderedScheduler;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -222,8 +223,8 @@
numReadErrors.set(0);
return true;
}
- if (BKException.Code.BookieHandleNotAvailableException == rc ||
- (isLongPoll && BKException.Code.NoSuchLedgerExistsException == rc)) {
+ if (BKException.Code.BookieHandleNotAvailableException == rc
+ || (isLongPoll && BKException.Code.NoSuchLedgerExistsException == rc)) {
int numErrors = Math.max(1, numReadErrors.incrementAndGet());
int nextReadBackoffTime = Math.min(numErrors * readAheadWaitTime, maxReadBackoffTime);
scheduler.schedule(
@@ -419,9 +420,9 @@
@Override
public synchronized void onLogSegmentMetadataUpdated(LogSegmentMetadata segment) {
- if (metadata == segment ||
- LogSegmentMetadata.COMPARATOR.compare(metadata, segment) == 0 ||
- !(metadata.isInProgress() && !segment.isInProgress())) {
+ if (metadata == segment
+ || LogSegmentMetadata.COMPARATOR.compare(metadata, segment) == 0
+ || !(metadata.isInProgress() && !segment.isInProgress())) {
return;
}
// segment is closed from inprogress, then re-open the log segment
@@ -474,7 +475,8 @@
if (isBeyondLastAddConfirmed()) {
// if the reader is already caught up, let's fail the reader immediately
// as we need to pull the latest metadata of this log segment.
- completeExceptionally(new BKTransmitException("Failed to open ledger for reading log segment " + getSegment(), rc),
+ completeExceptionally(new BKTransmitException("Failed to open ledger for reading log segment "
+ + getSegment(), rc),
true);
return;
}
@@ -581,8 +583,8 @@
if (cachedEntries >= maxPrefetchEntries) {
break;
}
- if ((isLedgerClosed() && nextEntryId > getLastAddConfirmed()) ||
- (!isLedgerClosed() && nextEntryId > getLastAddConfirmed() + 1)) {
+ if ((isLedgerClosed() && nextEntryId > getLastAddConfirmed())
+ || (!isLedgerClosed() && nextEntryId > getLastAddConfirmed() + 1)) {
break;
}
CacheEntry entry = new CacheEntry(nextEntryId);
@@ -688,7 +690,7 @@
}
/**
- * The core function to propagate fetched entries to read requests
+ * The core function to propagate fetched entries to read requests.
*/
@Override
public void run() {
@@ -777,13 +779,15 @@
CacheEntry removedEntry = readAheadEntries.poll();
try {
if (entry != removedEntry) {
- DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at reading from "
+ DLIllegalStateException ise =
+ new DLIllegalStateException("Unexpected condition at reading from "
+ getSegment());
completeExceptionally(ise, false);
return;
}
try {
- // the reference is retained on `entry.getEntry()`. Entry.Reader is responsible for releasing it.
+ // the reference is retained on `entry.getEntry()`.
+ // Entry.Reader is responsible for releasing it.
nextRequest.addEntry(processReadEntry(entry.getEntry()));
} catch (IOException e) {
completeExceptionally(e, false);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
index 14ebf4a..d53fcd9 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
@@ -17,7 +17,14 @@
*/
package org.apache.distributedlog.impl.logsegment;
+import static com.google.common.base.Charsets.UTF_8;
+import java.io.IOException;
import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.client.AsyncCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.distributedlog.BookKeeperClient;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.LogSegmentMetadata;
@@ -27,6 +34,7 @@
import org.apache.distributedlog.bk.LedgerAllocatorDelegator;
import org.apache.distributedlog.bk.QuorumConfigProvider;
import org.apache.distributedlog.bk.SimpleLedgerAllocator;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.exceptions.BKTransmitException;
import org.apache.distributedlog.injector.AsyncFailureInjector;
@@ -36,22 +44,15 @@
import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
import org.apache.distributedlog.metadata.LogMetadataForWriter;
import org.apache.distributedlog.util.Allocator;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.stats.StatsLogger;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import static com.google.common.base.Charsets.UTF_8;
/**
- * BookKeeper Based Entry Store
+ * BookKeeper Based Entry Store.
*/
public class BKLogSegmentEntryStore implements
LogSegmentEntryStore,
@@ -248,7 +249,8 @@
} catch (IOException e) {
return FutureUtils.exception(e);
}
- final CompletableFuture<LogSegmentRandomAccessEntryReader> openPromise = new CompletableFuture<LogSegmentRandomAccessEntryReader>();
+ final CompletableFuture<LogSegmentRandomAccessEntryReader> openPromise =
+ new CompletableFuture<LogSegmentRandomAccessEntryReader>();
AsyncCallback.OpenCallback openCallback = new AsyncCallback.OpenCallback() {
@Override
public void openComplete(int rc, LedgerHandle lh, Object ctx) {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
index 1f05359..51e21fd 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
@@ -19,9 +19,10 @@
import com.google.common.annotations.VisibleForTesting;
import io.netty.buffer.ByteBuf;
-import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
import org.apache.bookkeeper.client.AsyncCallback;
import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
+
/**
* Ledger based log segment entry writer.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
index 99ae6ad..3d50231 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
@@ -18,21 +18,23 @@
package org.apache.distributedlog.impl.logsegment;
import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.Enumeration;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.Entry;
-import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.exceptions.BKTransmitException;
-import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.Entry;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
-import java.io.IOException;
-import java.util.Enumeration;
-import java.util.List;
+
+
/**
* BookKeeper ledger based random access entry reader.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
index 82ba775..624d619 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
@@ -18,17 +18,18 @@
package org.apache.distributedlog.impl.logsegment;
import com.google.common.collect.Lists;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.common.functions.VoidFunctions;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.bookkeeper.client.AsyncCallback;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.functions.VoidFunctions;
-import java.util.List;
+
/**
- * BookKeeper Util Functions
+ * BookKeeper Util Functions.
*/
public class BKUtils {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/package-info.java
new file mode 100644
index 0000000..d3250bd
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * distributedlog logSegment impl.
+ */
+package org.apache.distributedlog.impl.logsegment;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java
index 1f3128d..4bcb474 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java
@@ -17,8 +17,16 @@
*/
package org.apache.distributedlog.impl.metadata;
+import static com.google.common.base.Charsets.UTF_8;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.DistributedLogConstants;
import org.apache.distributedlog.ZooKeeperClient;
@@ -32,13 +40,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URI;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import static com.google.common.base.Charsets.UTF_8;
/**
* Configurations for BookKeeper based DL.
@@ -58,8 +59,8 @@
dlConf.setCreateStreamIfNotExists(false);
LOG.info("Disabled createIfNotExists for federated namespace.");
}
- LOG.info("Propagate BKDLConfig to DLConfig : encodeRegionID = {}," +
- " firstLogSegmentSequenceNumber = {}, createStreamIfNotExists = {}, isFederated = {}.",
+ LOG.info("Propagate BKDLConfig to DLConfig : encodeRegionID = {},"
+ + " firstLogSegmentSequenceNumber = {}, createStreamIfNotExists = {}, isFederated = {}.",
new Object[] { dlConf.getEncodeRegionIDInLogSegmentMetadata(),
dlConf.getFirstLogSegmentSequenceNumber(), dlConf.getCreateStreamIfNotExists(),
bkdlConfig.isFederatedNamespace() });
@@ -75,7 +76,7 @@
}
}
assert (dlConfig instanceof BKDLConfig);
- return (BKDLConfig)dlConfig;
+ return (BKDLConfig) dlConfig;
}
@VisibleForTesting
@@ -225,7 +226,7 @@
/**
* Set the value at which ledger sequence number should start for streams that are being
* upgraded and did not have ledger sequence number to start with or for newly created
- * streams
+ * streams.
*
* @param firstLogSegmentSeqNo first ledger sequence number
* @return bk dl config
@@ -238,7 +239,7 @@
/**
* Get the value at which ledger sequence number should start for streams that are being
* upgraded and did not have ledger sequence number to start with or for newly created
- * streams
+ * streams.
*
* @return first ledger sequence number
*/
@@ -262,7 +263,7 @@
}
/**
- * Whether the namespace is federated namespace
+ * Whether the namespace is federated namespace.
*
* @return true if the namespace is a federated namespace. otherwise false.
*/
@@ -283,16 +284,16 @@
return false;
}
BKDLConfig another = (BKDLConfig) o;
- return Objects.equal(bkZkServersForWriter, another.bkZkServersForWriter) &&
- Objects.equal(bkZkServersForReader, another.bkZkServersForReader) &&
- Objects.equal(dlZkServersForWriter, another.dlZkServersForWriter) &&
- Objects.equal(dlZkServersForReader, another.dlZkServersForReader) &&
- Objects.equal(bkLedgersPath, another.bkLedgersPath) &&
- sanityCheckTxnID == another.sanityCheckTxnID &&
- encodeRegionID == another.encodeRegionID &&
- Objects.equal(aclRootPath, another.aclRootPath) &&
- Objects.equal(firstLogSegmentSeqNo, another.firstLogSegmentSeqNo) &&
- Objects.equal(isFederatedNamespace, another.isFederatedNamespace);
+ return Objects.equal(bkZkServersForWriter, another.bkZkServersForWriter)
+ && Objects.equal(bkZkServersForReader, another.bkZkServersForReader)
+ && Objects.equal(dlZkServersForWriter, another.dlZkServersForWriter)
+ && Objects.equal(dlZkServersForReader, another.dlZkServersForReader)
+ && Objects.equal(bkLedgersPath, another.bkLedgersPath)
+ && sanityCheckTxnID == another.sanityCheckTxnID
+ && encodeRegionID == another.encodeRegionID
+ && Objects.equal(aclRootPath, another.aclRootPath)
+ && Objects.equal(firstLogSegmentSeqNo, another.firstLogSegmentSeqNo)
+ && Objects.equal(isFederatedNamespace, another.isFederatedNamespace);
}
@@ -355,8 +356,8 @@
try {
configFormat.read(protocol);
} catch (TException e) {
- throw new IOException("Failed to deserialize data '" +
- new String(data, UTF_8) + "' : ", e);
+ throw new IOException("Failed to deserialize data '"
+ + new String(data, UTF_8) + "' : ", e);
}
// bookkeeper cluster settings
if (configFormat.isSetBkZkServers()) {
@@ -392,8 +393,8 @@
isFederatedNamespace = configFormat.isSetFederatedNamespace() && configFormat.isFederatedNamespace();
// Validate the settings
- if (null == bkZkServersForWriter || null == bkZkServersForReader || null == bkLedgersPath ||
- null == dlZkServersForWriter || null == dlZkServersForReader) {
+ if (null == bkZkServersForWriter || null == bkZkServersForReader || null == bkLedgersPath
+ || null == dlZkServersForWriter || null == dlZkServersForReader) {
throw new IOException("Missing zk/bk settings in BKDL Config : " + new String(data, UTF_8));
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
index 9b02462..04db205 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
@@ -17,15 +17,31 @@
*/
package org.apache.distributedlog.impl.metadata;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.distributedlog.metadata.LogMetadata.*;
+
import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
+
import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
import java.util.function.Function;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.versioning.Versioned;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.DistributedLogConstants;
import org.apache.distributedlog.ZooKeeperClient;
+
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.PermitManager;
+import org.apache.distributedlog.common.util.SchedulerUtils;
import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.exceptions.InvalidStreamNameException;
import org.apache.distributedlog.exceptions.LockCancelledException;
@@ -39,22 +55,20 @@
import org.apache.distributedlog.lock.ZKDistributedLock;
import org.apache.distributedlog.lock.ZKSessionLockFactory;
import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
-import org.apache.distributedlog.metadata.LogStreamMetadataStore;
import org.apache.distributedlog.metadata.LogMetadata;
import org.apache.distributedlog.metadata.LogMetadataForReader;
import org.apache.distributedlog.metadata.LogMetadataForWriter;
+import org.apache.distributedlog.metadata.LogStreamMetadataStore;
+
+
import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.common.util.SchedulerUtils;
-import org.apache.distributedlog.zk.LimitedPermitManager;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.common.util.PermitManager;
import org.apache.distributedlog.util.Transaction;
import org.apache.distributedlog.util.Utils;
+import org.apache.distributedlog.zk.LimitedPermitManager;
import org.apache.distributedlog.zk.ZKTransaction;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.versioning.Versioned;
+
+
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -68,19 +82,15 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.net.URI;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import static org.apache.distributedlog.metadata.LogMetadata.*;
+
/**
- * zookeeper based {@link LogStreamMetadataStore}
+ * zookeeper based {@link LogStreamMetadataStore}.
*/
public class ZKLogStreamMetadataStore implements LogStreamMetadataStore {
- private final static Logger LOG = LoggerFactory.getLogger(ZKLogStreamMetadataStore.class);
+ private static final Logger LOG = LoggerFactory.getLogger(ZKLogStreamMetadataStore.class);
private final String clientId;
private final DistributedLogConfiguration conf;
@@ -191,8 +201,9 @@
promise.completeExceptionally(new LogNotFoundException(
String.format("Log %s does not exist or has been deleted", logName)));
} else {
- promise.completeExceptionally(new ZKException("Error on checking log existence for " + logName,
- KeeperException.create(KeeperException.Code.get(rc))));
+ promise.completeExceptionally(
+ new ZKException("Error on checking log existence for "
+ + logName, KeeperException.create(KeeperException.Code.get(rc))));
}
}
}, null);
@@ -253,11 +264,13 @@
FutureUtils.complete(promise, null);
LOG.trace("Path {} is already existed.", path);
} else if (DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE == rc) {
- FutureUtils.completeExceptionally(promise, new ZooKeeperClient.ZooKeeperConnectionException(path));
+ FutureUtils.completeExceptionally(promise,
+ new ZooKeeperClient.ZooKeeperConnectionException(path));
} else if (DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE == rc) {
FutureUtils.completeExceptionally(promise, new DLInterruptedException(path));
} else {
- FutureUtils.completeExceptionally(promise, KeeperException.create(KeeperException.Code.get(rc)));
+ FutureUtils.completeExceptionally(promise,
+ KeeperException.create(KeeperException.Code.get(rc)));
}
}
}, null);
@@ -343,8 +356,8 @@
}
static void ensureMetadataExist(Versioned<byte[]> metadata) {
- Preconditions.checkNotNull(metadata.getValue());
- Preconditions.checkNotNull(metadata.getVersion());
+ checkNotNull(metadata.getValue());
+ checkNotNull(metadata.getVersion());
}
static void createMissingMetadata(final ZooKeeper zk,
@@ -487,7 +500,7 @@
// version
Versioned<byte[]> versionData = metadatas.get(MetadataIndex.VERSION);
ensureMetadataExist(maxTxnIdData);
- Preconditions.checkArgument(LAYOUT_VERSION == bytesToInt(versionData.getValue()));
+ checkArgument(LAYOUT_VERSION == bytesToInt(versionData.getValue()));
// lock path
ensureMetadataExist(metadatas.get(MetadataIndex.LOCK));
// read lock path
@@ -560,8 +573,9 @@
}
});
} catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- return FutureUtils.exception(new ZKException("Encountered zookeeper connection issue on creating log " + logName,
- KeeperException.Code.CONNECTIONLOSS));
+ return FutureUtils.exception(
+ new ZKException("Encountered zookeeper connection issue on creating log "
+ + logName, KeeperException.Code.CONNECTIONLOSS));
} catch (InterruptedException e) {
return FutureUtils.exception(new DLInterruptedException("Interrupted on creating log " + logName, e));
}
@@ -603,13 +617,15 @@
}
}, null);
} catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- FutureUtils.completeExceptionally(promise, new ZKException("Encountered zookeeper issue on deleting log stream "
- + logName, KeeperException.Code.CONNECTIONLOSS));
+ FutureUtils.completeExceptionally(promise,
+ new ZKException("Encountered zookeeper issue on deleting log stream "
+ + logName, KeeperException.Code.CONNECTIONLOSS));
} catch (InterruptedException e) {
- FutureUtils.completeExceptionally(promise, new DLInterruptedException("Interrupted while deleting log stream "
- + logName));
+ FutureUtils.completeExceptionally(promise,
+ new DLInterruptedException("Interrupted while deleting log stream " + logName));
} catch (KeeperException e) {
- FutureUtils.completeExceptionally(promise, new ZKException("Encountered zookeeper issue on deleting log stream "
+ FutureUtils.completeExceptionally(promise,
+ new ZKException("Encountered zookeeper issue on deleting log stream "
+ logName, e));
}
return promise;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZkMetadataResolver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZkMetadataResolver.java
index 1dee147..aaa5461 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZkMetadataResolver.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZkMetadataResolver.java
@@ -17,17 +17,20 @@
*/
package org.apache.distributedlog.impl.metadata;
+import java.io.IOException;
+import java.net.URI;
+import org.apache.commons.lang.StringUtils;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.metadata.DLMetadata;
import org.apache.distributedlog.metadata.MetadataResolver;
-import org.apache.commons.lang.StringUtils;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.common.PathUtils;
import org.apache.zookeeper.data.Stat;
-import java.io.IOException;
-import java.net.URI;
+/**
+ * zookeeper metadata resolver.
+ */
public class ZkMetadataResolver implements MetadataResolver {
private final ZooKeeperClient zkc;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/package-info.java
new file mode 100644
index 0000000..4e3cffd
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Metadata impl for distributedlog.
+ */
+package org.apache.distributedlog.impl.metadata;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
index 302c666..5354ac5 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
@@ -23,9 +23,10 @@
import java.util.concurrent.atomic.AtomicReference;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.api.subscription.SubscriptionStateStore;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+
+import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.util.Utils;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
@@ -33,10 +34,12 @@
import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
+/**
+ * The subscription state store Implementation.
+ */
public class ZKSubscriptionStateStore implements SubscriptionStateStore {
- static final Logger logger = LoggerFactory.getLogger(ZKSubscriptionStateStore.class);
+ private static final Logger logger = LoggerFactory.getLogger(ZKSubscriptionStateStore.class);
private final ZooKeeperClient zooKeeperClient;
private final String zkPath;
@@ -52,7 +55,7 @@
}
/**
- * Get the last committed position stored for this subscription
+ * Get the last committed position stored for this subscription.
*/
@Override
public CompletableFuture<DLSN> getLastCommitPosition() {
@@ -96,14 +99,14 @@
}
/**
- * Advances the position associated with the subscriber
+ * Advances the position associated with the subscriber.
*
* @param newPosition - new commit position
*/
@Override
public CompletableFuture<Void> advanceCommitPosition(DLSN newPosition) {
- if (null == lastCommittedPosition.get() ||
- (newPosition.compareTo(lastCommittedPosition.get()) > 0)) {
+ if (null == lastCommittedPosition.get()
+ || (newPosition.compareTo(lastCommittedPosition.get()) > 0)) {
lastCommittedPosition.set(newPosition);
return Utils.zkAsyncCreateFullPathOptimisticAndSetData(zooKeeperClient,
zkPath, newPosition.serialize().getBytes(Charsets.UTF_8),
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
index 0392264..89c3e12 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
@@ -29,10 +29,11 @@
import org.apache.commons.lang3.tuple.Pair;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.api.subscription.SubscriptionStateStore;
import org.apache.distributedlog.api.subscription.SubscriptionsStore;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+
import org.apache.distributedlog.util.Utils;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.KeeperException;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/package-info.java
new file mode 100644
index 0000000..f004b71
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * The subscription Implementation.
+ */
+package org.apache.distributedlog.impl.subscription;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncFailureInjector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncFailureInjector.java
index 6590bec..c748d6b 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncFailureInjector.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncFailureInjector.java
@@ -18,7 +18,7 @@
package org.apache.distributedlog.injector;
/**
- * Failure Injector that works in asynchronous way
+ * Failure Injector that works in asynchronous way.
*/
public interface AsyncFailureInjector {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncRandomFailureInjector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncRandomFailureInjector.java
index c6c675d..b054a67 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncRandomFailureInjector.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncRandomFailureInjector.java
@@ -17,12 +17,12 @@
*/
package org.apache.distributedlog.injector;
+import java.util.Random;
import org.apache.distributedlog.util.Utils;
-import java.util.Random;
/**
- * Failure injector based on {@link java.util.Random}
+ * Failure injector based on {@link java.util.Random}.
*/
public class AsyncRandomFailureInjector implements AsyncFailureInjector {
@@ -32,6 +32,9 @@
return new Builder();
}
+ /**
+ * Builder to build AsyncRandomFailureInjector.
+ */
public static class Builder {
private boolean _simulateDelays = false;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/FailureInjector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/FailureInjector.java
index 9544a6c..c1b8a98 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/FailureInjector.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/FailureInjector.java
@@ -25,7 +25,7 @@
/**
* No-op failure injector, which does nothing.
*/
- public static FailureInjector NULL = new FailureInjector() {
+ FailureInjector NULL = new FailureInjector() {
@Override
public void inject() {
// no-op;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java
index 2350b36..7fc567e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java
@@ -23,7 +23,7 @@
import org.slf4j.LoggerFactory;
/**
- * Injector that injects random delays
+ * Injector that injects random delays.
*/
public class RandomDelayFailureInjector implements FailureInjector {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/package-info.java
index d76dbc6..0e50f9b 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * DistributedLog Failure Injection
+ * DistributedLog Failure Injection.
*/
package org.apache.distributedlog.injector;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ChainedRequestLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ChainedRequestLimiter.java
index bbb4f7e..97bacb6 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ChainedRequestLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ChainedRequestLimiter.java
@@ -17,15 +17,17 @@
*/
package org.apache.distributedlog.limiter;
+
import com.google.common.base.Stopwatch;
import com.google.common.collect.ImmutableList;
-import org.apache.distributedlog.exceptions.OverCapacityException;
-
import java.util.concurrent.TimeUnit;
-
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+
+
+
/**
* Chain request limiters for easier management of multi limiter policy.
@@ -34,6 +36,9 @@
private final ImmutableList<RequestLimiter<Request>> limiters;
private final OpStatsLogger applyTime;
+ /**
+ * Builder to build ChainedRequestLimiter.
+ */
public static class Builder<Request> {
private final ImmutableList.Builder<RequestLimiter<Request>> limitersBuilder;
private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
index ae01bf7..08f0135 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
@@ -17,7 +17,7 @@
*/
package org.apache.distributedlog.limiter;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.bookkeeper.stats.Counter;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.distributedlog.exceptions.OverCapacityException;
@@ -25,31 +25,37 @@
import org.slf4j.LoggerFactory;
/**
- * Collect rate limiter implementation, cost(Request), overlimit, etc. behavior.
+ * Collect rate limiter implementation, cost(RequestT), overlimit, etc. behavior.
*/
-public class ComposableRequestLimiter<Request> implements RequestLimiter<Request> {
+public class ComposableRequestLimiter<RequestT> implements RequestLimiter<RequestT> {
protected static final Logger LOG = LoggerFactory.getLogger(ComposableRequestLimiter.class);
private final RateLimiter limiter;
- private final OverlimitFunction<Request> overlimitFunction;
- private final CostFunction<Request> costFunction;
+ private final OverlimitFunction<RequestT> overlimitFunction;
+ private final CostFunction<RequestT> costFunction;
private final Counter overlimitCounter;
- static public interface OverlimitFunction<Request> {
- void apply(Request request) throws OverCapacityException;
+ /**
+ * OverlimitFunction.
+ */
+ public interface OverlimitFunction<RequestT> {
+ void apply(RequestT request) throws OverCapacityException;
}
- static public interface CostFunction<Request> {
- int apply(Request request);
+ /**
+ * CostFunction.
+ */
+ public interface CostFunction<RequestT> {
+ int apply(RequestT request);
}
public ComposableRequestLimiter(
RateLimiter limiter,
- OverlimitFunction<Request> overlimitFunction,
- CostFunction<Request> costFunction,
+ OverlimitFunction<RequestT> overlimitFunction,
+ CostFunction<RequestT> costFunction,
StatsLogger statsLogger) {
- Preconditions.checkNotNull(limiter);
- Preconditions.checkNotNull(overlimitFunction);
- Preconditions.checkNotNull(costFunction);
+ checkNotNull(limiter);
+ checkNotNull(overlimitFunction);
+ checkNotNull(costFunction);
this.limiter = limiter;
this.overlimitFunction = overlimitFunction;
this.costFunction = costFunction;
@@ -57,7 +63,7 @@
}
@Override
- public void apply(Request request) throws OverCapacityException {
+ public void apply(RequestT request) throws OverCapacityException {
int permits = costFunction.apply(request);
if (!limiter.acquire(permits)) {
overlimitCounter.inc();
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/GuavaRateLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/GuavaRateLimiter.java
index f18e5a3..953f959 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/GuavaRateLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/GuavaRateLimiter.java
@@ -17,7 +17,7 @@
*/
package org.apache.distributedlog.limiter;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkState;
/**
* Wrap a guava limiter in a simple interface to make testing easier.
@@ -48,7 +48,7 @@
@Override
public boolean acquire(int permits) {
- Preconditions.checkState(permits >= 0);
+ checkState(permits >= 0);
if (permits > 0) {
return limiter.tryAcquire(permits);
} else {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RateLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RateLimiter.java
index d1ddf0d..55f6fd0 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RateLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RateLimiter.java
@@ -22,21 +22,23 @@
*/
public interface RateLimiter {
- public static final RateLimiter REJECT = new RateLimiter() {
+ RateLimiter REJECT = new RateLimiter() {
@Override
public boolean acquire(int permits) {
return false;
}
};
- public static final RateLimiter ACCEPT = new RateLimiter() {
+ RateLimiter ACCEPT = new RateLimiter() {
@Override
public boolean acquire(int permits) {
return true;
}
};
-
- public static abstract class Builder {
+ /**
+ * Builder for a rate limiter.
+ */
+ abstract class Builder {
public abstract RateLimiter build();
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RequestLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RequestLimiter.java
index 73bc953..e59014b 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RequestLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RequestLimiter.java
@@ -18,7 +18,9 @@
package org.apache.distributedlog.limiter;
import org.apache.distributedlog.exceptions.OverCapacityException;
-
-public interface RequestLimiter<Request> {
- public void apply(Request request) throws OverCapacityException;
+/**
+ * RequestLimiter.
+ */
+public interface RequestLimiter<RequestT> {
+ void apply(RequestT request) throws OverCapacityException;
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/package-info.java
index b41ad6d..3c0d4b5 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * Rate limiting for distributedlog
+ * Rate limiting for distributedlog.
*/
package org.apache.distributedlog.limiter;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLock.java
index 156d6dd..6fe85df 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLock.java
@@ -22,7 +22,7 @@
import org.apache.distributedlog.io.AsyncCloseable;
/**
- * Interface for distributed locking
+ * Interface for distributed locking.
*/
public interface DistributedLock extends AsyncCloseable {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLockContext.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLockContext.java
index 4f84bb4..8642214 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLockContext.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLockContext.java
@@ -16,11 +16,11 @@
* limitations under the License.
*/
package org.apache.distributedlog.lock;
-
-import org.apache.commons.lang3.tuple.Pair;
-
import java.util.HashSet;
import java.util.Set;
+import org.apache.commons.lang3.tuple.Pair;
+
+
class DistributedLockContext {
private final Set<Pair<String, Long>> lockIds;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockAction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockAction.java
index 767088f..11d8e83 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockAction.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockAction.java
@@ -18,12 +18,12 @@
package org.apache.distributedlog.lock;
/**
- * Lock Action
+ * Lock Action.
*/
interface LockAction {
/**
- * Execute a lock action
+ * Execute a lock action.
*/
void execute();
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockClosedException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockClosedException.java
index fe6b254..22115ae 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockClosedException.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockClosedException.java
@@ -16,10 +16,9 @@
* limitations under the License.
*/
package org.apache.distributedlog.lock;
-
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.lock.ZKSessionLock.State;
-import org.apache.commons.lang3.tuple.Pair;
/**
* Exception indicates that the lock was closed (unlocked) before the lock request could complete.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockListener.java
index 2a528e9..4810cf1 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockListener.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockListener.java
@@ -18,7 +18,7 @@
package org.apache.distributedlog.lock;
/**
- * Listener on lock state changes
+ * Listener on lock state changes.
*/
interface LockListener {
/**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockSessionExpiredException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockSessionExpiredException.java
index 62fef20..b02c92d 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockSessionExpiredException.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockSessionExpiredException.java
@@ -16,10 +16,9 @@
* limitations under the License.
*/
package org.apache.distributedlog.lock;
-
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.lock.ZKSessionLock.State;
-import org.apache.commons.lang3.tuple.Pair;
/**
* Exception indicates that the lock's zookeeper session was expired before the lock request could complete.
@@ -29,6 +28,7 @@
private static final long serialVersionUID = 8775253025963470331L;
public LockSessionExpiredException(String lockPath, Pair<String, Long> lockId, State currentState) {
- super(lockPath, "lock at path " + lockPath + " with id " + lockId + " expired early in state : " + currentState);
+ super(lockPath, "lock at path " + lockPath + " with id "
+ + lockId + " expired early in state : " + currentState);
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockStateChangedException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockStateChangedException.java
index 362f602..71d26ed 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockStateChangedException.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockStateChangedException.java
@@ -17,12 +17,12 @@
*/
package org.apache.distributedlog.lock;
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.lock.ZKSessionLock.State;
-import org.apache.commons.lang3.tuple.Pair;
/**
- * Exception thrown when lock state changed
+ * Exception thrown when lock state changed.
*/
public class LockStateChangedException extends LockingException {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockTimeoutException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockTimeoutException.java
index 33f3582..56336f4 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockTimeoutException.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockTimeoutException.java
@@ -17,12 +17,12 @@
*/
package org.apache.distributedlog.lock;
+import java.util.concurrent.TimeUnit;
import org.apache.distributedlog.exceptions.LockingException;
-import java.util.concurrent.TimeUnit;
/**
- * Exception thrown when acquiring lock timeout
+ * Exception thrown when acquiring lock timeout.
*/
public class LockTimeoutException extends LockingException {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java
index 1cb3364..123c437 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockWaiter.java
@@ -20,8 +20,8 @@
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Utils;
import org.slf4j.Logger;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
index 7f770ad..79f5ae2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
@@ -18,8 +18,8 @@
package org.apache.distributedlog.lock;
import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.LockingException;
/**
* An implementation of {@link DistributedLock} which does nothing.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLock.java
index 3a46a13..9abfac9 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLock.java
@@ -24,16 +24,19 @@
/**
* One time lock.
- * <p>The lock is only alive during a given period. It should
+ *
+ * <p>The lock is only alive during a given period. It should
* be not usable if the lock is expired.
- * <p>Listener could be registered by {@link #setLockListener(LockListener)}
+ *
+ * <p>Listener could be registered by {@link #setLockListener(LockListener)}
* to receive state changes of the lock.
*/
public interface SessionLock {
/**
* Set lock listener for lock state changes.
- * <p>Typically a listener should be set before try locking.
+ *
+ * <p>Typically a listener should be set before try locking.
*
* @param lockListener
* lock listener for state changes.
@@ -49,7 +52,8 @@
/**
* Whether the lock is expired or not?
- * <p>If a lock is expired, it will not be reusable any more. Because it is an one-time lock.
+ *
+ * <p>If a lock is expired, it will not be reusable any more. Because it is an one-time lock.
*
* @return true if the lock is expired, otherwise false.
*/
@@ -57,19 +61,19 @@
/**
* Acquire the lock if it is free within given waiting time.
- * <p>
- * Calling this method will attempt to acquire the lock. If the lock
+ *
+ * <p>Calling this method will attempt to acquire the lock. If the lock
* is already acquired by others, the caller will wait for <i>timeout</i>
* period. If the caller could claim the lock within <i>timeout</i> period,
* the caller acquire the lock. Otherwise, it would fail with {@link OwnershipAcquireFailedException}.
- * <p>
- * {@link #unlock()} should be called to unlock a claimed lock. The caller
+ *
+ * <p>{@link #unlock()} should be called to unlock a claimed lock. The caller
* doesn't need to unlock to clean up resources if <i>tryLock</i> fails.
- * <p>
- * <i>tryLock</i> here is effectively the combination of following asynchronous calls.
+ *
+ * <p><i>tryLock</i> here is effectively the combination of following asynchronous calls.
* <pre>
* ZKDistributedLock lock = ...;
- * CompletableFuture<LockWaiter> attemptFuture = lock.asyncTryLock(...);
+ * CompletableFuture<LockWaiter> attemptFuture = lock.asyncTryLock(...);
*
* boolean acquired = waiter.waitForAcquireQuietly();
* if (acquired) {
@@ -91,8 +95,8 @@
/**
* Acquire the lock in asynchronous way.
- * <p>
- * Calling this method will attempt to place a lock waiter to acquire this lock.
+ *
+ * <p>Calling this method will attempt to place a lock waiter to acquire this lock.
* The future returned by this method represents the result of this attempt. It doesn't mean
* the caller acquired the lock or not. The application should check {@link LockWaiter#getAcquireFuture()}
* to see if it acquired the lock or not.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLockFactory.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLockFactory.java
index 9d3159e..05c1f8f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLockFactory.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLockFactory.java
@@ -20,7 +20,7 @@
import java.util.concurrent.CompletableFuture;
/**
- * Factory to create {@link SessionLock}
+ * Factory to create {@link SessionLock}.
*/
public interface SessionLockFactory {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
index 664d0bc..fbe4eb2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
@@ -28,12 +28,12 @@
import org.apache.bookkeeper.stats.Counter;
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.distributedlog.exceptions.LockingException;
-import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import org.apache.distributedlog.exceptions.UnexpectedException;
import org.apache.distributedlog.common.concurrent.AsyncSemaphore;
import org.apache.distributedlog.common.concurrent.FutureEventListener;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
import org.apache.distributedlog.util.OrderedScheduler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -50,8 +50,8 @@
* The lock is only allowed to acquire once. If the lock is acquired successfully,
* the caller holds the ownership until it loses the ownership either because of
* others already acquired the lock when session expired or explicitly close it.
- * <p>
- * The caller could use {@link #checkOwnership()} or {@link #checkOwnershipAndReacquire()}
+ *
+ * <p>The caller could use {@link #checkOwnership()} or {@link #checkOwnershipAndReacquire()}
* to check if it still holds the lock. If it doesn't hold the lock, the caller should
* give up the ownership and close the lock.
* <h3>Metrics</h3>
@@ -134,7 +134,8 @@
*/
public synchronized CompletableFuture<ZKDistributedLock> asyncAcquire() {
if (null != lockAcquireFuture) {
- return FutureUtils.exception(new UnexpectedException("Someone is already acquiring/acquired lock " + lockPath));
+ return FutureUtils.exception(
+ new UnexpectedException("Someone is already acquiring/acquired lock " + lockPath));
}
final CompletableFuture<ZKDistributedLock> promise = FutureUtils.createFuture();
promise.whenComplete((zkDistributedLock, throwable) -> {
@@ -474,7 +475,8 @@
doAsyncAcquireWithSemaphore(tryPromise, 0);
}
- private CompletableFuture<ZKDistributedLock> reacquireLock(boolean throwLockAcquireException) throws LockingException {
+ private CompletableFuture<ZKDistributedLock> reacquireLock(boolean throwLockAcquireException)
+ throws LockingException {
final Stopwatch stopwatch = Stopwatch.createStarted();
CompletableFuture<ZKDistributedLock> lockPromise;
synchronized (this) {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
index 1325751..d9f3b2c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
@@ -33,24 +33,25 @@
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
-import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.exceptions.LockingException;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.common.stats.OpStatsListener;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.bookkeeper.stats.Counter;
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.util.SafeRunnable;
import org.apache.commons.lang3.tuple.Pair;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+
+import org.apache.distributedlog.common.stats.OpStatsListener;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -66,15 +67,15 @@
* It is not reusable: if lock failed, if zookeeper session is expired, if #unlock is called,
* it would be transitioned to expired or closed state.
*
- * The Locking Procedure is described as below.
+ *<p>The Locking Procedure is described as below.</p>
*
* <p>
- * 0. if it is an immediate lock, it would get lock waiters first. if the lock is already held
- * by someone. it would fail immediately with {@link org.apache.distributedlog.exceptions.OwnershipAcquireFailedException}
+ * 0. if it is an immediate lock, it would get lock waiters first. if the lock is already held by someone.
+ * it would fail immediately with {@link org.apache.distributedlog.exceptions.OwnershipAcquireFailedException}
* with current owner. if there is no lock waiters, it would start locking procedure from 1.
* 1. prepare: create a sequential znode to identify the lock.
- * 2. check lock waiters: get all lock waiters to check after prepare. if it is the first waiter,
- * claim the ownership; if it is not the first waiter, but first waiter was itself (same client id and same session id)
+ * 2. check lock waiters: get all lock waiters to check after prepare. if it is the first waiter, claim the ownership;
+ * if it is not the first waiter, but first waiter was itself (same client id and same session id)
* claim the ownership too; otherwise, it would set watcher on its sibling and wait it to disappared.
* </p>
*
@@ -141,11 +142,12 @@
return lockPath + LOCK_PATH_PREFIX + URLEncoder.encode(clientId, UTF_8.name()) + LOCK_PART_SEP;
}
- public static String getLockPathPrefixV3(String lockPath, String clientId, long sessionOwner) throws UnsupportedEncodingException {
+ public static String getLockPathPrefixV3(String lockPath, String clientId, long sessionOwner)
+ throws UnsupportedEncodingException {
// member_<clientid>_s<owner_session>_
StringBuilder sb = new StringBuilder();
- sb.append(lockPath).append(LOCK_PATH_PREFIX).append(URLEncoder.encode(clientId, UTF_8.name())).append(LOCK_PART_SEP)
- .append("s").append(String.format("%10d", sessionOwner)).append(LOCK_PART_SEP);
+ sb.append(lockPath).append(LOCK_PATH_PREFIX).append(URLEncoder.encode(clientId, UTF_8.name())).
+ append(LOCK_PART_SEP).append("s").append(String.format("%10d", sessionOwner)).append(LOCK_PART_SEP);
return sb.toString();
}
@@ -174,7 +176,7 @@
}
};
- static enum State {
+ enum State {
INIT, // initialized state
PREPARING, // preparing to lock, but no lock node created
PREPARED, // lock node created
@@ -235,8 +237,8 @@
}
private boolean isTryingOrClaimed() {
- return inState(State.PREPARING) || inState(State.PREPARED) ||
- inState(State.WAITING) || inState(State.CLAIMED);
+ return inState(State.PREPARING) || inState(State.PREPARED)
+ || inState(State.WAITING) || inState(State.CLAIMED);
}
public boolean isExpiredOrClosing() {
@@ -404,8 +406,10 @@
}
} else {
if (LOG.isTraceEnabled()) {
- LOG.trace("{} skipped executing lock action '{}' for lock {}, since epoch is changed from {} to {}.",
- new Object[]{lockId, func.getActionName(), lockPath, lockEpoch, ZKSessionLock.this.epoch.get()});
+ LOG.trace("{} skipped executing lock action '{}' for lock {},"
+ + " since epoch is changed from {} to {}.",
+ new Object[]{lockId, func.getActionName(),
+ lockPath, lockEpoch, ZKSessionLock.this.epoch.get()});
}
}
}
@@ -424,7 +428,8 @@
* @param promise
* promise
*/
- protected <T> void executeLockAction(final int lockEpoch, final LockAction func, final CompletableFuture<T> promise) {
+ protected <T> void executeLockAction(final int lockEpoch,
+ final LockAction func, final CompletableFuture<T> promise) {
lockStateExecutor.submit(lockPath, new SafeRunnable() {
@Override
public void safeRun() {
@@ -441,7 +446,8 @@
}
} else {
if (LOG.isTraceEnabled()) {
- LOG.trace("{} skipped executing lock action '{}' for lock {}, since epoch is changed from {} to {}.",
+ LOG.trace("{} skipped executing lock action '{}' for lock {},"
+ + " since epoch is changed from {} to {}.",
new Object[]{lockId, func.getActionName(), lockPath, lockEpoch, currentEpoch});
}
promise.completeExceptionally(new EpochChangedException(lockPath, lockEpoch, currentEpoch));
@@ -451,7 +457,7 @@
}
/**
- * Parse member id generated by zookeeper from given <i>nodeName</i>
+ * Parse member id generated by zookeeper from given <i>nodeName</i>.
*
* @param nodeName
* lock node name
@@ -507,7 +513,8 @@
* node name
* @return client id and its ephemeral owner.
*/
- static CompletableFuture<Pair<String, Long>> asyncParseClientID(ZooKeeper zkClient, String lockPath, String nodeName) {
+ static CompletableFuture<Pair<String, Long>> asyncParseClientID(
+ ZooKeeper zkClient, String lockPath, String nodeName) {
String[] parts = nodeName.split("_");
// member_<clientid>_s<owner_session>_
if (4 == parts.length && parts[2].startsWith("s")) {
@@ -550,7 +557,8 @@
@Override
public void safeRun() {
if (!lockState.inState(State.INIT)) {
- result.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
+ result.completeExceptionally(new LockStateChangedException(lockPath,
+ lockId, State.INIT, lockState.getState()));
return;
}
if (KeeperException.Code.OK.intValue() != rc) {
@@ -627,7 +635,7 @@
});
}
}
- });;
+ });
return new LockWaiter(
lockId.getLeft(),
currentOwner,
@@ -653,7 +661,8 @@
@Override
public void execute() {
if (!lockState.inState(State.INIT)) {
- result.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
+ result.completeExceptionally(
+ new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
return;
}
asyncTryLock(false, result);
@@ -686,8 +695,8 @@
public void onFailure(final Throwable lockCause) {
// If tryLock failed due to state changed, we don't need to cleanup
if (lockCause instanceof LockStateChangedException) {
- LOG.info("skipping cleanup for {} at {} after encountering lock " +
- "state change exception : ", new Object[] { lockId, lockPath, lockCause });
+ LOG.info("skipping cleanup for {} at {} after encountering lock "
+ + "state change exception : ", new Object[] { lockId, lockPath, lockCause });
result.completeExceptionally(lockCause);
return;
}
@@ -718,8 +727,8 @@
* the sibling is dead. <i>acquireCompletableFuture</i> will be notified either it locked successfully
* or the lock failed. The promise will only satisfy with current lock owner.
*
- * NOTE: the <i>promise</i> is only satisfied on <i>lockStateExecutor</i>, so any
- * transformations attached on promise will be executed in order.
+ * <p>NOTE: the <i>promise</i> is only satisfied on <i>lockStateExecutor</i>, so any
+ * transformations attached on promise will be executed in order.</p>
*
* @param wait
* whether to wait for ownership.
@@ -731,7 +740,8 @@
@Override
public void execute() {
if (!lockState.inState(State.INIT)) {
- promise.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
+ promise.completeExceptionally(
+ new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
return;
}
lockState.transition(State.PREPARING);
@@ -748,8 +758,8 @@
} catch (UnsupportedEncodingException uee) {
myPath = getLockPathPrefixV1(lockPath);
}
- zk.create(myPath, serializeClientId(lockId.getLeft()), zkClient.getDefaultACL(), CreateMode.EPHEMERAL_SEQUENTIAL,
- new AsyncCallback.StringCallback() {
+ zk.create(myPath, serializeClientId(lockId.getLeft()), zkClient.getDefaultACL(),
+ CreateMode.EPHEMERAL_SEQUENTIAL, new AsyncCallback.StringCallback() {
@Override
public void processResult(final int rc, String path, Object ctx, final String name) {
executeLockAction(curEpoch, new LockAction() {
@@ -761,7 +771,8 @@
return;
}
- if (FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition)) {
+ if (FailpointUtils.checkFailPointNoThrow(
+ FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition)) {
lockState.transition(State.CLOSING);
lockState.transition(State.CLOSED);
}
@@ -776,8 +787,8 @@
LOG.trace("{} received member id for lock {}", lockId, currentId);
if (lockState.isExpiredOrClosing()) {
- // Delete node attempt may have come after PREPARING but before create node, in which case
- // we'd be left with a dangling node unless we clean up.
+ // Delete node attempt may have come after PREPARING but before create node,
+ // in which case we'd be left with a dangling node unless we clean up.
CompletableFuture<Void> deletePromise = new CompletableFuture<Void>();
deleteLockNode(deletePromise);
FutureUtils.ensure(
@@ -973,8 +984,8 @@
public void safeRun() {
if (KeeperException.Code.OK.intValue() == rc) {
LOG.info("Deleted lock node {} for {} successfully.", path, lockId);
- } else if (KeeperException.Code.NONODE.intValue() == rc ||
- KeeperException.Code.SESSIONEXPIRED.intValue() == rc) {
+ } else if (KeeperException.Code.NONODE.intValue() == rc
+ || KeeperException.Code.SESSIONEXPIRED.intValue() == rc) {
LOG.info("Delete node failed. Node already gone for node {} id {}, rc = {}",
new Object[] { path, lockId, KeeperException.Code.get(rc) });
} else {
@@ -1117,7 +1128,8 @@
@Override
public void execute() {
if (!lockState.inState(State.PREPARED)) { // e.g. lock closed or session expired after prepared
- promise.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
+ promise.completeExceptionally(
+ new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
return;
}
@@ -1145,7 +1157,8 @@
promise.complete(cid);
} else if (memberIndex > 0) { // we are in the member list but we didn't hold the lock
// get ownership of current owner
- asyncParseClientID(zk, lockPath, children.get(0)).whenComplete(new FutureEventListener<Pair<String, Long>>() {
+ asyncParseClientID(zk, lockPath, children.get(0)).whenComplete(
+ new FutureEventListener<Pair<String, Long>>() {
@Override
public void onSuccess(Pair<String, Long> currentOwner) {
watchLockOwner(lockWatcher, wait,
@@ -1171,8 +1184,8 @@
LOG.error("Member {} doesn't exist in the members list {} for lock {}.",
new Object[]{ cid, children, lockPath});
promise.completeExceptionally(
- new UnexpectedException("Member " + cid + " doesn't exist in member list " +
- children + " for lock " + lockPath));
+ new UnexpectedException("Member " + cid + " doesn't exist in member list "
+ + children + " for lock " + lockPath));
}
}
@@ -1218,20 +1231,22 @@
// we should watch it and claim ownership
shouldWatch = true;
shouldClaimOwnership = true;
- LOG.info("LockWatcher {} for {} found its previous session {} held lock, watch it to claim ownership.",
- new Object[] { myNode, lockPath, currentOwner });
+ LOG.info("LockWatcher {} for {} found its previous session {} held lock,"
+ + " watch it to claim ownership.", new Object[] { myNode, lockPath, currentOwner });
} else if (lockId.compareTo(currentOwner) == 0 && areLockWaitersInSameSession(siblingNode, ownerNode)) {
// I found that my sibling is the current owner with same lock id (client id & session id)
// It must be left by any race condition from same zookeeper client
shouldWatch = true;
shouldClaimOwnership = true;
- LOG.info("LockWatcher {} for {} found itself {} already held lock at sibling node {}, watch it to claim ownership.",
+ LOG.info("LockWatcher {} for {} found itself {} already held lock at sibling node {},"
+ + " watch it to claim ownership.",
new Object[]{myNode, lockPath, lockId, siblingNode});
} else {
shouldWatch = wait;
if (wait) {
if (LOG.isDebugEnabled()) {
- LOG.debug("Current LockWatcher for {} with ephemeral node {}, is waiting for {} to release lock at {}.",
+ LOG.debug("Current LockWatcher for {} with ephemeral node {}, "
+ + "is waiting for {} to release lock at {}.",
new Object[]{lockPath, myNode, siblingNode, System.currentTimeMillis()});
}
}
@@ -1248,7 +1263,8 @@
@Override
public void execute() {
if (!lockState.inState(State.PREPARED)) {
- promise.completeExceptionally(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
+ promise.completeExceptionally(new LockStateChangedException(lockPath,
+ lockId, State.PREPARED, lockState.getState()));
return;
}
@@ -1268,7 +1284,8 @@
// sibling just disappeared, it might be the chance to claim ownership
checkLockOwnerAndWaitIfPossible(lockWatcher, wait, promise);
} else {
- promise.completeExceptionally(KeeperException.create(KeeperException.Code.get(rc)));
+ promise.completeExceptionally(
+ KeeperException.create(KeeperException.Code.get(rc)));
}
}
@@ -1309,7 +1326,7 @@
@Override
public void process(WatchedEvent event) {
LOG.debug("Received event {} from lock {} at {} : watcher epoch {}, lock epoch {}.",
- new Object[] { event, lockPath, System.currentTimeMillis(), epoch, ZKSessionLock.this.epoch.get() });
+ new Object[] {event, lockPath, System.currentTimeMillis(), epoch, ZKSessionLock.this.epoch.get() });
if (event.getType() == Watcher.Event.EventType.None) {
switch (event.getState()) {
case SyncConnected:
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
index 223a3a4..6c592f4 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
@@ -23,11 +23,11 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.DLInterruptedException;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.util.OrderedScheduler;
/**
* Factory to create zookeeper based locks.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/package-info.java
index baeaab2..e5c15ab 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * Distributed locking mechanism in distributedlog
+ * Distributed locking mechanism in distributedlog.
*/
package org.apache.distributedlog.lock;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryReader.java
index 1b292e3..eef5882 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryReader.java
@@ -26,11 +26,13 @@
/**
* An interface class to read the enveloped entry (serialized bytes of
- * {@link org.apache.distributedlog.Entry}) from a log segment
+ * {@link org.apache.distributedlog.Entry}) from a log segment.
*/
@Beta
public interface LogSegmentEntryReader extends AsyncCloseable {
-
+ /**
+ * An interface Listener for StateChange.
+ */
interface StateChangeListener {
/**
@@ -47,7 +49,7 @@
void start();
/**
- * Register the state change listener
+ * Register the state change listener.
*
* @param listener register the state change listener
* @return entry reader
@@ -55,7 +57,7 @@
LogSegmentEntryReader registerListener(StateChangeListener listener);
/**
- * Unregister the state change listener
+ * Unregister the state change listener.
*
* @param listener register the state change listener
* @return entry reader
@@ -78,12 +80,12 @@
/**
* Read next <i>numEntries</i> entries from current log segment.
- * <p>
- * <i>numEntries</i> will be best-effort.
+ *
+ * <p><i>numEntries</i> will be best-effort.
*
* @param numEntries num entries to read from current log segment
* @return A promise that when satisified will contain a non-empty list of entries with their content.
- * @throws {@link org.apache.distributedlog.exceptions.EndOfLogSegmentException} when
+ * @throw {@link org.apache.distributedlog.exceptions.EndOfLogSegmentException} when
* read entries beyond the end of a <i>closed</i> log segment.
*/
CompletableFuture<List<Entry.Reader>> readNext(int numEntries);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java
index ab2d898..4822b7e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java
@@ -18,16 +18,16 @@
package org.apache.distributedlog.logsegment;
import com.google.common.annotations.Beta;
+import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import org.apache.distributedlog.LogSegmentMetadata;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.metadata.LogMetadataForWriter;
import org.apache.distributedlog.util.Allocator;
-import java.io.IOException;
/**
- * Log Segment Store to read log segments
+ * Log Segment Store to read log segments.
*/
@Beta
public interface LogSegmentEntryStore {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryWriter.java
index 5ded597..d10091b 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryWriter.java
@@ -19,9 +19,9 @@
import com.google.common.annotations.Beta;
import io.netty.buffer.ByteBuf;
+import org.apache.bookkeeper.client.AsyncCallback;
import org.apache.distributedlog.Entry;
import org.apache.distributedlog.common.util.Sizable;
-import org.apache.bookkeeper.client.AsyncCallback;
/**
* An interface class to write the enveloped entry (serialized bytes of
@@ -51,7 +51,8 @@
/**
* Async add entry to the log segment.
- * <p>The implementation semantic follows
+ *
+ * <p>The implementation semantic follows
* {@link org.apache.bookkeeper.client.LedgerHandle#asyncAddEntry(
* byte[], int, int, AsyncCallback.AddCallback, Object)}
*
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentFilter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentFilter.java
index 6a6d5e5..ffbfb14 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentFilter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentFilter.java
@@ -20,11 +20,11 @@
import java.util.Collection;
/**
- * Filter to filter log segments
+ * Filter to filter log segments.
*/
public interface LogSegmentFilter {
- public static final LogSegmentFilter DEFAULT_FILTER = new LogSegmentFilter() {
+ LogSegmentFilter DEFAULT_FILTER = new LogSegmentFilter() {
@Override
public Collection<String> filter(Collection<String> fullList) {
return fullList;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java
index abc0c74..4d99587 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java
@@ -17,20 +17,21 @@
*/
package org.apache.distributedlog.logsegment;
+
import com.google.common.base.Ticker;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
+import java.util.concurrent.TimeUnit;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.LogSegmentMetadata;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.concurrent.TimeUnit;
/**
- * Cache the log segment metadata
+ * Cache the log segment metadata.
*/
public class LogSegmentMetadataCache implements RemovalListener<String, LogSegmentMetadata> {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataStore.java
index a58264c..8597d84 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataStore.java
@@ -18,6 +18,8 @@
package org.apache.distributedlog.logsegment;
import com.google.common.annotations.Beta;
+import java.io.Closeable;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.versioning.Version;
import org.apache.bookkeeper.versioning.Versioned;
@@ -28,8 +30,6 @@
import org.apache.distributedlog.util.Transaction;
import org.apache.distributedlog.util.Transaction.OpListener;
-import java.io.Closeable;
-import java.util.List;
/**
* Interface for log segment metadata store. All operations that modify log segments should
@@ -67,7 +67,7 @@
OpListener<Version> listener);
/**
- * Store the maximum transaction id for <code>path</code>
+ * Store the maximum transaction id for <code>path</code>.
*
* @param txn
* transaction to execute for storing transaction id
@@ -86,8 +86,8 @@
/**
* Create a log segment <code>segment</code> under transaction <code>txn</code>.
*
- * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
- * immediately. the operation should be executed via {@link Transaction#execute()}
+ * <p>NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
+ * immediately. the operation should be executed via {@link Transaction#execute()}</p>
*
* @param txn
* transaction to execute for this operation
@@ -103,8 +103,8 @@
/**
* Delete a log segment <code>segment</code> under transaction <code>txn</code>.
*
- * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
- * immediately. the operation should be executed via {@link Transaction#execute()}
+ * <p>NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
+ * immediately. the operation should be executed via {@link Transaction#execute()}</p>
*
* @param txn
* transaction to execute for this operation
@@ -118,8 +118,8 @@
/**
* Update a log segment <code>segment</code> under transaction <code>txn</code>.
*
- * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
- * immediately. the operation should be executed via {@link Transaction#execute()}
+ * <p>NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
+ * immediately. the operation should be executed via {@link Transaction#execute()}</p>
*
* @param txn
* transaction to execute for this operation
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentWriter.java
index c483403..6deb205 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentWriter.java
@@ -38,7 +38,7 @@
*
* @return log segment id.
*/
- public long getLogSegmentId();
+ long getLogSegmentId();
/**
* Write a log record to a log segment.
@@ -52,12 +52,11 @@
* @throws BKTransmitException if failed to transmit data to bk
* @throws org.apache.distributedlog.exceptions.WriteException if failed to write to bk
*/
- public CompletableFuture<DLSN> asyncWrite(LogRecord record);
+ CompletableFuture<DLSN> asyncWrite(LogRecord record);
/**
* This isn't a simple synchronous version of {@code asyncWrite}. It has different semantic.
* This method only writes data to the buffer and flushes buffer if needed.
- *
* TODO: we should remove this method. when we rewrite synchronous writer based on asynchronous writer,
* since this is the semantic needed to be provided in higher level but just calling write & flush.
*
@@ -65,7 +64,7 @@
* @throws IOException when tried to flush the buffer.
* @see LogSegmentWriter#asyncWrite(LogRecord)
*/
- public void write(LogRecord record) throws IOException;
+ void write(LogRecord record) throws IOException;
/**
* Transmit the buffered data and wait for it being persisted and return the last acknowledged
@@ -73,7 +72,7 @@
*
* @return future representing the transmit result with last acknowledged transaction id.
*/
- public CompletableFuture<Long> flush();
+ CompletableFuture<Long> flush();
/**
* Commit the current acknowledged data. It is the consequent operation of {@link #flush()},
@@ -81,6 +80,6 @@
*
* @return future representing the commit result.
*/
- public CompletableFuture<Long> commit();
+ CompletableFuture<Long> commit();
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java
index 7929505..a2796c2 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java
@@ -19,13 +19,6 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Sets;
-import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
@@ -35,6 +28,14 @@
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
/**
* Managing log segments in local cache.
@@ -70,8 +71,7 @@
/**
* Retrieve log segments from the cache.
- *
- * - first sort the log segments in ascending order
+ *- first sort the log segments in ascending order
* - do validation and assign corresponding sequence id
* - apply comparator after validation
*
@@ -96,13 +96,16 @@
LogSegmentMetadata segment = segmentsToReturn.get(i);
if (null != prevSegment
- && prevSegment.getVersion() >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
- && segment.getVersion() >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
+ && prevSegment.getVersion()
+ >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
+ && segment.getVersion()
+ >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
&& prevSegment.getLogSegmentSequenceNumber() + 1 != segment.getLogSegmentSequenceNumber()) {
LOG.error("{} found ledger sequence number gap between log segment {} and {}",
new Object[] { streamName, prevSegment, segment });
throw new UnexpectedException(streamName + " found ledger sequence number gap between log segment "
- + prevSegment.getLogSegmentSequenceNumber() + " and " + segment.getLogSegmentSequenceNumber());
+ + prevSegment.getLogSegmentSequenceNumber()
+ + " and " + segment.getLogSegmentSequenceNumber());
}
prevSegment = segment;
}
@@ -127,7 +130,8 @@
} else {
if (segment.supportsSequenceId()) {
LogSegmentMetadata newSegment = segment.mutator()
- .setStartSequenceId(startSequenceId == DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ? 0L : startSequenceId)
+ .setStartSequenceId(startSequenceId
+ == DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ? 0L : startSequenceId)
.build();
segmentsToReturn.set(i, newSegment);
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/RollingPolicy.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/RollingPolicy.java
index 4844ad4..cf7c266 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/RollingPolicy.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/RollingPolicy.java
@@ -18,7 +18,10 @@
package org.apache.distributedlog.logsegment;
import org.apache.distributedlog.common.util.Sizable;
-
+/**
+ * Policy for rolling.
+ *
+ */
public interface RollingPolicy {
/**
* Determines if a rollover may be appropriate at this time.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/SizeBasedRollingPolicy.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/SizeBasedRollingPolicy.java
index 1c3794a..17ddb16 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/SizeBasedRollingPolicy.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/SizeBasedRollingPolicy.java
@@ -18,7 +18,10 @@
package org.apache.distributedlog.logsegment;
import org.apache.distributedlog.common.util.Sizable;
-
+/**
+ * SizeBased Policy for rolling.
+ *
+ */
public class SizeBasedRollingPolicy implements RollingPolicy {
final long maxSize;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java
index 80c09be..a3ee01b 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java
@@ -21,10 +21,13 @@
import org.apache.distributedlog.util.Utils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
+/**
+ * TimeBased Policy for rolling.
+ *
+ */
public class TimeBasedRollingPolicy implements RollingPolicy {
- final static Logger LOG = LoggerFactory.getLogger(TimeBasedRollingPolicy.class);
+ static final Logger LOG = LoggerFactory.getLogger(TimeBasedRollingPolicy.class);
final long rollingIntervalMs;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/package-info.java
index 5b0c497..a79e40a 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * Log Segment Management
+ * Log Segment Management.
*/
package org.apache.distributedlog.logsegment;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLConfig.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLConfig.java
index f9cd27b..948c3af 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLConfig.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLConfig.java
@@ -20,13 +20,13 @@
import java.io.IOException;
/**
- * Specific config of a given implementation of DL
+ * Specific config of a given implementation of DL.
*/
public interface DLConfig {
/**
* Serialize the dl config into a string.
*/
- public String serialize();
+ String serialize();
/**
* Deserialize the dl config from a readable stream.
@@ -35,5 +35,5 @@
* bytes to desrialize dl config.
* @throws IOException if fail to deserialize the dl config string representation.
*/
- public void deserialize(byte[] data) throws IOException;
+ void deserialize(byte[] data) throws IOException;
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java
index 948f2bf..0bb2079 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.java
@@ -17,23 +17,24 @@
*/
package org.apache.distributedlog.metadata;
+import static com.google.common.base.Charsets.UTF_8;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.StringReader;
+import java.net.URI;
import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
import org.apache.distributedlog.exceptions.ZKException;
import org.apache.distributedlog.impl.metadata.BKDLConfig;
import org.apache.distributedlog.util.Utils;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.ZooKeeperClientBuilder;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.StringReader;
-import java.net.URI;
-import static com.google.common.base.Charsets.UTF_8;
+
/**
* Metadata of a given DL instance.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
index fe52804..db84983 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
@@ -19,10 +19,13 @@
import java.util.concurrent.CompletableFuture;
import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
import org.apache.distributedlog.util.Transaction;
+/**
+ * Class to update the zookeeper-based log metadata store.
+ */
public class DryrunLogSegmentMetadataStoreUpdater extends LogSegmentMetadataStoreUpdater {
public DryrunLogSegmentMetadataStoreUpdater(DistributedLogConfiguration conf,
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadata.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadata.java
index b78ebcf..c9f1beb 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadata.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadata.java
@@ -20,7 +20,7 @@
import java.net.URI;
/**
- * Class to represent the layout and metadata of the zookeeper-based log metadata
+ * Class to represent the layout and metadata of the zookeeper-based log metadata.
*/
public class LogMetadata {
@@ -70,14 +70,14 @@
}
public static final int LAYOUT_VERSION = -1;
- public final static String LOGSEGMENTS_PATH = "/ledgers";
- public final static String VERSION_PATH = "/version";
+ public static final String LOGSEGMENTS_PATH = "/ledgers";
+ public static final String VERSION_PATH = "/version";
// writer znodes
- public final static String MAX_TXID_PATH = "/maxtxid";
- public final static String LOCK_PATH = "/lock";
- public final static String ALLOCATION_PATH = "/allocation";
+ public static final String MAX_TXID_PATH = "/maxtxid";
+ public static final String LOCK_PATH = "/lock";
+ public static final String ALLOCATION_PATH = "/allocation";
// reader znodes
- public final static String READ_LOCK_PATH = "/readLock";
+ public static final String READ_LOCK_PATH = "/readLock";
protected final URI uri;
protected final String logName;
@@ -92,7 +92,7 @@
protected final String allocationPath;
/**
- * metadata representation of a log
+ * metadata representation of a log.
*
* @param uri
* namespace to store the log
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java
index 1adebfe..6f5f865 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java
@@ -22,7 +22,7 @@
import java.net.URI;
/**
- * Log Metadata for Reader
+ * Log Metadata for Reader.
*/
public class LogMetadataForReader extends LogMetadata {
@@ -73,10 +73,10 @@
return new LogMetadataForReader(uri, logName, logIdentifier);
}
- final static String SUBSCRIBERS_PATH = "/subscribers";
+ static final String SUBSCRIBERS_PATH = "/subscribers";
/**
- * metadata representation of a log
+ * metadata representation of a log.
*
* @param uri namespace to store the log
* @param logName name of the log
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForWriter.java
index 35e645e..e63983c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForWriter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForWriter.java
@@ -17,12 +17,12 @@
*/
package org.apache.distributedlog.metadata;
+import java.net.URI;
import org.apache.bookkeeper.versioning.Versioned;
-import java.net.URI;
/**
- * Log Metadata for writer
+ * Log Metadata for writer.
*/
public class LogMetadataForWriter extends LogMetadata {
@@ -31,7 +31,7 @@
private final Versioned<byte[]> allocationData;
/**
- * metadata representation of a log
+ * metadata representation of a log.
*
* @param uri namespace to store the log
* @param logName name of the log
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
index 71a1f98..aef60ff 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
@@ -17,7 +17,8 @@
*/
package org.apache.distributedlog.metadata;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
import java.util.concurrent.CompletableFuture;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.DistributedLogConfiguration;
@@ -28,6 +29,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+/**
+ *The implementation is responsible
+ * for updating the metadata.
+ */
public class LogSegmentMetadataStoreUpdater implements MetadataUpdater {
static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadataStoreUpdater.class);
@@ -59,9 +64,9 @@
public CompletableFuture<LogSegmentMetadata> updateLastRecord(LogSegmentMetadata segment,
LogRecordWithDLSN record) {
DLSN dlsn = record.getDlsn();
- Preconditions.checkState(!segment.isInProgress(),
+ checkState(!segment.isInProgress(),
"Updating last dlsn for an inprogress log segment isn't supported.");
- Preconditions.checkArgument(segment.isDLSNinThisSegment(dlsn),
+ checkArgument(segment.isDLSNinThisSegment(dlsn),
"DLSN " + dlsn + " doesn't belong to segment " + segment);
final LogSegmentMetadata newSegment = segment.mutator()
.setLastDLSN(dlsn)
@@ -85,7 +90,7 @@
}
/**
- * Change the truncation status of a <i>log segment</i> to be active
+ * Change the truncation status of a <i>log segment</i> to be active.
*
* @param segment log segment to change truncation status to active.
* @return new log segment
@@ -99,7 +104,7 @@
}
/**
- * Change the truncation status of a <i>log segment</i> to truncated
+ * Change the truncation status of a <i>log segment</i> to truncated.
*
* @param segment log segment to change truncation status to truncated.
* @return new log segment
@@ -122,14 +127,15 @@
}
/**
- * Change the truncation status of a <i>log segment</i> to partially truncated
+ * Change the truncation status of a <i>log segment</i> to partially truncated.
*
* @param segment log segment to change sequence number.
* @param minActiveDLSN DLSN within the log segment before which log has been truncated
* @return new log segment
*/
@Override
- public CompletableFuture<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment, DLSN minActiveDLSN) {
+ public CompletableFuture<LogSegmentMetadata>
+ setLogSegmentPartiallyTruncated(LogSegmentMetadata segment, DLSN minActiveDLSN) {
final LogSegmentMetadata newSegment = segment.mutator()
.setTruncationStatus(LogSegmentMetadata.TruncationStatus.PARTIALLY_TRUNCATED)
.setMinActiveDLSN(minActiveDLSN)
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java
index 37ecab4..41dc500 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java
@@ -19,14 +19,15 @@
import com.google.common.annotations.Beta;
import com.google.common.base.Optional;
-import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.lock.DistributedLock;
-import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
-import org.apache.distributedlog.common.util.PermitManager;
-import org.apache.distributedlog.util.Transaction;
-
import java.io.Closeable;
import java.net.URI;
+import java.util.concurrent.CompletableFuture;
+import org.apache.distributedlog.common.util.PermitManager;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
+import org.apache.distributedlog.util.Transaction;
+
+
/**
* The interface to manage the log stream metadata. The implementation is responsible
@@ -43,7 +44,7 @@
Transaction<Object> newTransaction();
/**
- * Ensure the existence of a log stream
+ * Ensure the existence of a log stream.
*
* @param uri the location of the log stream
* @param logName the name of the log stream
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataResolver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataResolver.java
index ae0002e..09c4a5e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataResolver.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataResolver.java
@@ -47,5 +47,5 @@
* @return dl metadata.
* @throws IOException
*/
- public DLMetadata resolve(URI uri) throws IOException;
+ DLMetadata resolve(URI uri) throws IOException;
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataUpdater.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataUpdater.java
index 793a2c9..0a036c6 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataUpdater.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataUpdater.java
@@ -29,7 +29,7 @@
public interface MetadataUpdater {
/**
- * Start a transaction on metadata updates
+ * Start a transaction on metadata updates.
*
* @return transaction
*/
@@ -60,7 +60,7 @@
long logSegmentSeqNo);
/**
- * Change the truncation status of a <i>log segment</i> to be active
+ * Change the truncation status of a <i>log segment</i> to be active.
*
* @param segment
* log segment to change truncation status to active.
@@ -69,7 +69,7 @@
CompletableFuture<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment);
/**
- * Change the truncation status of a <i>log segment</i> to truncated
+ * Change the truncation status of a <i>log segment</i> to truncated.
*
* @param segment
* log segment to change truncation status to truncated.
@@ -90,7 +90,7 @@
LogSegmentMetadata setLogSegmentTruncated(Transaction<Object> txn, LogSegmentMetadata segment);
/**
- * Change the truncation status of a <i>log segment</i> to partially truncated
+ * Change the truncation status of a <i>log segment</i> to partially truncated.
*
* @param segment
* log segment to change sequence number.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
index cf970ef..432b22a 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
@@ -17,29 +17,33 @@
*/
package org.apache.distributedlog.namespace;
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URI;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.api.MetadataAccessor;
import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.api.MetadataAccessor;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
import org.apache.distributedlog.exceptions.InvalidStreamNameException;
import org.apache.distributedlog.injector.AsyncFailureInjector;
import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
import org.apache.distributedlog.metadata.LogMetadataStore;
import org.apache.distributedlog.metadata.LogStreamMetadataStore;
-import org.apache.distributedlog.api.subscription.SubscriptionsStore;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.URI;
+
+
/**
* Manager to manage all the stores required by a namespace.
*/
public interface NamespaceDriver extends Closeable {
-
+ /**
+ * Role associated with the store.
+ */
enum Role {
WRITER,
READER
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java
index b9e3a31..e580c8b 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java
@@ -17,21 +17,21 @@
*/
package org.apache.distributedlog.namespace;
+import static com.google.common.base.Preconditions.*;
import com.google.common.base.Objects;
import com.google.common.collect.Sets;
-import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import java.net.URI;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
-import static com.google.common.base.Preconditions.*;
+
/**
* The basic service for managing a set of namespace drivers.
@@ -111,7 +111,6 @@
* Register the namespace {@code driver}.
*
* @param driver the namespace driver
- * @return the namespace driver manager
*/
public static void registerDriver(String backend, Class<? extends NamespaceDriver> driver) {
if (!initialized) {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceWatcher.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceWatcher.java
index 7c9dda2..703f9cc 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceWatcher.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceWatcher.java
@@ -17,9 +17,9 @@
*/
package org.apache.distributedlog.namespace;
+import java.util.concurrent.CopyOnWriteArraySet;
import org.apache.distributedlog.callback.NamespaceListener;
-import java.util.concurrent.CopyOnWriteArraySet;
/**
* Namespace Watcher watching namespace changes.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/package-info.java
index 5b86dbe..ddd79c6 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * DistributedLog Namespace
+ * DistributedLog Namespace.
*/
package org.apache.distributedlog.namespace;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolver.java
index b6a0652..7170cf9 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolver.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolver.java
@@ -17,15 +17,16 @@
*/
package org.apache.distributedlog.net;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
import org.apache.commons.lang.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
+
/**
* Abstract DNS resolver for bookkeeper ensemble placement.
@@ -48,8 +49,8 @@
/**
* Construct the dns resolver with host-region overrides.
- * <p>
- * <i>hostRegionOverrides</i> is a string of pairs of host-region mapping
+ *
+ * <p><i>hostRegionOverrides</i> is a string of pairs of host-region mapping
* (host:region) separated by ';'. during dns resolution, the host will be resolved
* to override region. example: <i>host1:region1;host2:region2;...</i>
*
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRacks.java b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRacks.java
index 3408077..27a3631 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRacks.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRacks.java
@@ -19,16 +19,16 @@
/**
* Resolve the dns by racks.
- * <p>
- * It resolves domain name like `(region)-(rack)-xxx-xxx.*` to network location
+ *
+ * <p>It resolves domain name like `(region)-(rack)-xxx-xxx.*` to network location
* `/(region)/(rack)`. If resolution failed, it returns `/default-region/default-rack`.
- * <p>
- * region could be override in <code>hostRegionOverrides</code>. for example, if the
+ *
+ * <p>region could be override in <code>hostRegionOverrides</code>. for example, if the
* host name is <i>regionA-rack1-xx-yyy</i>, it would be resolved to `/regionA/rack1`
* without any overrides. If the specified overrides is <i>regionA-rack1-xx-yyy:regionB</i>,
* the resolved network location would be <i>/regionB/rack1</i>.
- * <p>
- * Region overrides provide optimization hits to bookkeeper if two `logical` regions are
+ *
+ * <p>Region overrides provide optimization hits to bookkeeper if two `logical` regions are
* in same or close locations.
*
* @see DNSResolver#DNSResolver(String)
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRows.java b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRows.java
index ec604f2..f2a6be8 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRows.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRows.java
@@ -19,16 +19,16 @@
/**
* Resolve the dns by rows.
- * <p>
- * It resolves domain name like `(region)-(row)xx-xxx-xxx.*` to network location
+ *
+ * <p>It resolves domain name like `(region)-(row)xx-xxx-xxx.*` to network location
* `/(region)/(row)`. If resolution failed, it returns `/default-region/default-row`.
- * <p>
- * region could be override in <code>hostRegionOverrides</code>. for example, if the
+ *
+ * <p>region could be override in <code>hostRegionOverrides</code>. for example, if the
* host name is <i>regionA-row1-xx-yyy</i>, it would be resolved to `/regionA/row1`
* without any overrides. If the specified overrides is <i>regionA-row1-xx-yyy:regionB</i>,
* the resolved network location would be <i>/regionB/row1</i>.
- * <p>
- * Region overrides provide optimization hits to bookkeeper if two `logical` regions are
+ *
+ * <p>Region overrides provide optimization hits to bookkeeper if two `logical` regions are
* in same or close locations.
*
* @see DNSResolver#DNSResolver(String)
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/net/NetUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/net/NetUtils.java
index 765980e..13f0fe6 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/net/NetUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/net/NetUtils.java
@@ -17,21 +17,22 @@
*/
package org.apache.distributedlog.net;
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
+import org.apache.bookkeeper.net.DNSToSwitchMapping;
+
+
/**
- * Utils about network
+ * Utils about network.
*/
public class NetUtils {
/**
* Get the dns resolver from class <code>resolverClassName</code> with optional
* <code>hostRegionOverrides</code>.
- * <p>
- * It would try to load the class with the constructor with <code>hostRegionOverrides</code>.
+ *
+ * <p>It would try to load the class with the constructor with <code>hostRegionOverrides</code>.
* If it fails, it would fall back to load the class with default empty constructor.
* The interpretion of <code>hostRegionOverrides</code> is up to the implementation.
*
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/package-info.java
index 34a42ea..984acfa 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * Protocol & Core of DistributedLog
+ * Protocol & Core of DistributedLog.
*/
package org.apache.distributedlog;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstRecordSelector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstRecordSelector.java
index b6a6adf..3247a6c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstRecordSelector.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstRecordSelector.java
@@ -20,7 +20,7 @@
import org.apache.distributedlog.LogRecordWithDLSN;
/**
- * Save the first record processed
+ * Save the first record processed.
*/
public class FirstRecordSelector implements LogRecordSelector {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/selector/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/package-info.java
new file mode 100644
index 0000000..be9ad7f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * selector for selecting specific log record.
+ */
+package org.apache.distributedlog.selector;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
index e50a172..04c0810 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
@@ -17,6 +17,10 @@
*/
package org.apache.distributedlog.tools;
+import static com.google.common.base.Charsets.UTF_8;
+import static com.google.common.base.Preconditions.checkArgument;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
import io.netty.buffer.ByteBuf;
import java.io.BufferedReader;
import java.io.File;
@@ -50,27 +54,12 @@
import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
-
-import com.google.common.base.Preconditions;
-import org.apache.distributedlog.BKDistributedLogNamespace;
-import org.apache.distributedlog.Entry;
-import org.apache.distributedlog.api.MetadataAccessor;
-import org.apache.distributedlog.api.namespace.Namespace;
-import org.apache.distributedlog.callback.NamespaceListener;
-import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.Utils;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.BookKeeperAccessor;
import org.apache.bookkeeper.client.BookKeeperAdmin;
import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
import org.apache.bookkeeper.client.LedgerReader;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
@@ -81,40 +70,51 @@
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.configuration.ConfigurationException;
import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.RateLimiter;
-import org.apache.distributedlog.api.AsyncLogReader;
-import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.BKDistributedLogNamespace;
import org.apache.distributedlog.BookKeeperClient;
import org.apache.distributedlog.BookKeeperClientBuilder;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.exceptions.LogNotFoundException;
-import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.Entry;
import org.apache.distributedlog.LogRecord;
import org.apache.distributedlog.LogRecordWithDLSN;
import org.apache.distributedlog.LogSegmentMetadata;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.api.MetadataAccessor;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.auditor.DLAuditor;
import org.apache.distributedlog.bk.LedgerAllocator;
import org.apache.distributedlog.bk.LedgerAllocatorUtils;
+import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.metadata.MetadataUpdater;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.common.util.SchedulerUtils;
+import org.apache.distributedlog.metadata.MetadataUpdater;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
-import static com.google.common.base.Charsets.UTF_8;
-public class DistributedLogTool extends Tool {
- static final Logger logger = LoggerFactory.getLogger(DistributedLogTool.class);
+
+/**
+ *DistributedLogTool.
+*/
+ public class DistributedLogTool extends Tool {
+
+ private static final Logger logger = LoggerFactory.getLogger(DistributedLogTool.class);
static final List<String> EMPTY_LIST = Lists.newArrayList();
@@ -212,7 +212,8 @@
} 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.");
+ throw new ParseException("Failed to load distributedlog configuration from "
+ + configFile + ": malformed uri.");
}
}
if (cmdline.hasOption("a")) {
@@ -304,9 +305,9 @@
return runSimpleCmd();
}
- abstract protected int runSimpleCmd() throws Exception;
+ protected abstract int runSimpleCmd() throws Exception;
- abstract protected void parseCommandLine(CommandLine cmdline) throws ParseException;
+ protected abstract void parseCommandLine(CommandLine cmdline) throws ParseException;
@Override
protected Options getOptions() {
@@ -365,13 +366,15 @@
if (cmdline.hasOption("t")) {
concurrency = Integer.parseInt(cmdline.getOptionValue("t"));
if (concurrency <= 0) {
- throw new ParseException("Invalid concurrency value : " + concurrency + ": it must be greater or equal to 0.");
+ throw new ParseException("Invalid concurrency value : "
+ + concurrency + ": it must be greater or equal to 0.");
}
}
if (cmdline.hasOption("ap")) {
allocationPoolPath = cmdline.getOptionValue("ap");
if (!allocationPoolPath.startsWith(".") || !allocationPoolPath.contains("allocation")) {
- throw new ParseException("Invalid allocation pool path : " + allocationPoolPath + ": it must starts with a '.' and must contains 'allocation'");
+ throw new ParseException("Invalid allocation pool path : "
+ + allocationPoolPath + ": it must starts with a '.' and must contains 'allocation'");
}
}
}
@@ -381,7 +384,7 @@
String rootPath = getUri().getPath() + "/" + allocationPoolPath;
final ScheduledExecutorService allocationExecutor = Executors.newSingleThreadScheduledExecutor();
ExecutorService executorService = Executors.newFixedThreadPool(concurrency);
- Preconditions.checkArgument(getNamespace() instanceof BKDistributedLogNamespace);
+ checkArgument(getNamespace() instanceof BKDistributedLogNamespace);
BKDistributedLogNamespace bkns = (BKDistributedLogNamespace) getNamespace();
final ZooKeeperClient zkc = ((BKNamespaceDriver) bkns.getNamespaceDriver()).getWriterZKC();
final BookKeeperClient bkc = ((BKNamespaceDriver) bkns.getNamespaceDriver()).getReaderBKC();
@@ -411,7 +414,8 @@
allocator.delete();
System.out.println("Deleted allocator pool : " + poolPath + " .");
} catch (IOException ioe) {
- System.err.println("Failed to delete allocator pool " + poolPath + " : " + ioe.getMessage());
+ System.err.println("Failed to delete allocator pool "
+ + poolPath + " : " + ioe.getMessage());
}
}
doneLatch.countDown();
@@ -434,6 +438,9 @@
}
}
+/**
+ * List distributedlog associated info.
+ */
public static class ListCommand extends PerDLCommand {
boolean printMetadata = false;
@@ -489,7 +496,9 @@
System.out.println("--------------------------------");
}
}
-
+/**
+ * watch and report changes for a dl namespace.
+ */
public static class WatchNamespaceCommand extends PerDLCommand implements NamespaceListener {
private Set<String> currentSet = Sets.<String>newHashSet();
private CountDownLatch doneLatch = new CountDownLatch(1);
@@ -539,7 +548,9 @@
namespace.registerNamespaceListener(this);
}
}
-
+/**
+ * Inspect streams under a given dl uri to find any potential corruptions.
+ */
protected static class InspectCommand extends PerDLCommand {
int numThreads = 1;
@@ -587,7 +598,8 @@
System.out.println(corruptedCandidates.keySet());
return 0;
}
- for (Map.Entry<String, List<Pair<LogSegmentMetadata, List<String>>>> entry : corruptedCandidates.entrySet()) {
+ for (Map.Entry<String, List<Pair<LogSegmentMetadata,
+ List<String>>>> entry : corruptedCandidates.entrySet()) {
System.out.println(entry.getKey() + " : \n");
for (Pair<LogSegmentMetadata, List<String>> pair : entry.getValue()) {
System.out.println("\t - " + pair.getLeft());
@@ -604,8 +616,8 @@
return 0;
}
- private void inspectStreams(final SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates)
- throws Exception {
+ private void inspectStreams(final SortedMap<String,
+ List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates) throws Exception {
Iterator<String> streamCollection = getNamespace().getLogs();
final List<String> streams = new ArrayList<String>();
while (streamCollection.hasNext()) {
@@ -622,7 +634,8 @@
return;
}
println("Streams : " + streams);
- if (!getForce() && !IOUtils.confirmPrompt("Are you sure you want to inspect " + streams.size() + " streams")) {
+ if (!getForce() && !IOUtils.confirmPrompt("Are you sure you want to inspect "
+ + streams.size() + " streams")) {
return;
}
numThreads = Math.min(streams.size(), numThreads);
@@ -673,7 +686,8 @@
}
}
for (LogSegmentMetadata segment : segments) {
- if (!segment.isInProgress() && inprogressSeqNos.contains(segment.getLogSegmentSequenceNumber())) {
+ if (!segment.isInProgress()
+ && inprogressSeqNos.contains(segment.getLogSegmentSequenceNumber())) {
isCandidate = true;
}
}
@@ -700,8 +714,8 @@
LogSegmentMetadata segment = seg;
List<String> dumpedEntries = new ArrayList<String>();
if (segment.isInProgress()) {
- LedgerHandle lh = bkc.get().openLedgerNoRecovery(segment.getLogSegmentId(), BookKeeper.DigestType.CRC32,
- dlConf.getBKDigestPW().getBytes(UTF_8));
+ LedgerHandle lh = bkc.get().openLedgerNoRecovery(segment.getLogSegmentId(),
+ BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes(UTF_8));
try {
long lac = lh.readLastConfirmed();
segment = segment.mutator().setLastEntryId(lac).build();
@@ -740,6 +754,9 @@
}
}
+ /**
+ * Command used to truncate streams under a given dl uri.
+ */
protected static class TruncateCommand extends PerDLCommand {
int numThreads = 1;
@@ -846,6 +863,9 @@
}
}
+ /**
+ * Simple bk client.
+ */
public static class SimpleBookKeeperClient {
BookKeeperClient bkc;
ZooKeeperClient zkc;
@@ -882,6 +902,9 @@
}
}
+ /**
+ * Command used to show metadata of a given stream and list segments.
+ */
protected static class ShowCommand extends PerStreamCommand {
SimpleBookKeeperClient bkc = null;
@@ -962,7 +985,8 @@
long firstTxnId = dlm.getFirstTxId();
long lastTxnId = dlm.getLastTxId();
long recordCount = dlm.getLogRecordCount();
- String result = String.format("Stream : (firstTxId=%d, lastTxid=%d, firstDlsn=%s, lastDlsn=%s, endOfStreamMarked=%b, recordCount=%d)",
+ String result = String.format("Stream : (firstTxId=%d, lastTxid=%d, firstDlsn=%s,"
+ + " lastDlsn=%s, endOfStreamMarked=%b, recordCount=%d)",
firstTxnId, lastTxnId, getDlsnName(firstDlsn), getDlsnName(lastDlsn), endOfStreamMarked, recordCount);
System.out.println(result);
if (listEppStats) {
@@ -971,7 +995,8 @@
}
boolean include(LogSegmentMetadata segment) {
- return (firstLid <= segment.getLogSegmentSequenceNumber() && (lastLid == -1 || lastLid >= segment.getLogSegmentSequenceNumber()));
+ return (firstLid <= segment.getLogSegmentSequenceNumber()
+ && (lastLid == -1 || lastLid >= segment.getLogSegmentSequenceNumber()));
}
private void printEppStatsHeader(DistributedLogManager dlm) throws Exception {
@@ -984,10 +1009,12 @@
merge(totals, getBookieStats(segment));
}
}
- List<Map.Entry<BookieSocketAddress, Integer>> entries = new ArrayList<Map.Entry<BookieSocketAddress, Integer>>(totals.entrySet());
+ List<Map.Entry<BookieSocketAddress, Integer>> entries =
+ new ArrayList<Map.Entry<BookieSocketAddress, Integer>>(totals.entrySet());
Collections.sort(entries, new Comparator<Map.Entry<BookieSocketAddress, Integer>>() {
@Override
- public int compare(Map.Entry<BookieSocketAddress, Integer> o1, Map.Entry<BookieSocketAddress, Integer> o2) {
+ public int compare(Map.Entry<BookieSocketAddress, Integer> o1,
+ Map.Entry<BookieSocketAddress, Integer> o2) {
return o2.getValue() - o1.getValue();
}
});
@@ -998,7 +1025,8 @@
totalEntries += entry.getValue();
}
for (Map.Entry<BookieSocketAddress, Integer> entry : entries) {
- System.out.println(String.format("%"+width+"s\t%6.2f%%\t\t%d", entry.getKey(), entry.getValue()*1.0/totalEntries, entry.getValue()));
+ System.out.println(String.format("%" + width + "s\t%6.2f%%\t\t%d",
+ entry.getKey(), entry.getValue() * 1.0 / totalEntries, entry.getValue()));
}
}
@@ -1008,10 +1036,11 @@
private Map<BookieSocketAddress, Integer> getBookieStats(LogSegmentMetadata segment) throws Exception {
Map<BookieSocketAddress, Integer> stats = new HashMap<BookieSocketAddress, Integer>();
- LedgerHandle lh = bkc.client().get().openLedgerNoRecovery(segment.getLogSegmentId(), BookKeeper.DigestType.CRC32,
- getConf().getBKDigestPW().getBytes(UTF_8));
+ LedgerHandle lh = bkc.client().get().openLedgerNoRecovery(segment.getLogSegmentId(),
+ BookKeeper.DigestType.CRC32, getConf().getBKDigestPW().getBytes(UTF_8));
long eidFirst = 0;
- for (SortedMap.Entry<Long, ArrayList<BookieSocketAddress>> entry : LedgerReader.bookiesForLedger(lh).entrySet()) {
+ for (SortedMap.Entry<Long, ArrayList<BookieSocketAddress>>
+ entry : LedgerReader.bookiesForLedger(lh).entrySet()) {
long eidLast = entry.getKey().longValue();
long count = eidLast - eidFirst + 1;
for (BookieSocketAddress bookie : entry.getValue()) {
@@ -1129,6 +1158,10 @@
}
}
+
+ /**
+ * Command used to delete a given stream.
+ */
public static class DeleteCommand extends PerStreamCommand {
protected DeleteCommand() {
@@ -1153,6 +1186,9 @@
}
}
+ /**
+ * Command used to delete given ledgers.
+ */
public static class DeleteLedgersCommand extends PerDLCommand {
private final List<Long> ledgers = new ArrayList<Long>();
@@ -1173,7 +1209,8 @@
throw new ParseException("Please specify ledgers: either use list or use file only.");
}
if (!cmdline.hasOption("l") && !cmdline.hasOption("lf")) {
- throw new ParseException("No ledgers specified. Please specify ledgers either use list or use file only.");
+ throw new ParseException("No ledgers specified."
+ + " Please specify ledgers either use list or use file only.");
}
if (cmdline.hasOption("l")) {
String ledgersStr = cmdline.getOptionValue("l");
@@ -1268,6 +1305,9 @@
}
}
+ /**
+ * Command used to create streams under a given namespace.
+ */
public static class CreateCommand extends PerDLCommand {
final List<String> streams = new ArrayList<String>();
@@ -1278,8 +1318,8 @@
CreateCommand() {
super("create", "create streams under a given namespace");
options.addOption("r", "prefix", true, "Prefix of stream name. E.g. 'QuantumLeapTest-'.");
- options.addOption("e", "expression", true, "Expression to generate stream suffix. " +
- "Currently we support range 'x-y', list 'x,y,z' and name 'xyz'");
+ options.addOption("e", "expression", true, "Expression to generate stream suffix. "
+ + "Currently we support range 'x-y', list 'x,y,z' and name 'xyz'");
}
@Override
@@ -1362,7 +1402,9 @@
this.streamExpression = expression;
}
}
-
+ /**
+ * Command used to dump records of a given stream.
+ */
protected static class DumpCommand extends PerStreamCommand {
boolean printHex = false;
@@ -1452,8 +1494,8 @@
return 0;
}
try {
- System.out.println(String.format("Dump records for %s (from = %s, dump count = %d, total records = %d)",
- getStreamName(), startOffset, count, totalCount));
+ System.out.println(String.format("Dump records for %s (from = %s, dump"
+ + " count = %d, total records = %d)", getStreamName(), startOffset, count, totalCount));
dumpRecords(reader);
} finally {
@@ -1519,9 +1561,8 @@
}
/**
- * TODO: refactor inspect & inspectstream
+ * TODO: refactor inspect & inspectstream.
* TODO: support force
- *
* inspectstream -lac -gap (different options for different operations for a single stream)
* inspect -lac -gap (inspect the namespace, which will use inspect stream)
*/
@@ -1608,7 +1649,8 @@
final AtomicInteger rcHolder = new AtomicInteger(-1234);
final CountDownLatch doneLatch = new CountDownLatch(1);
try {
- lr.forwardReadEntriesFromLastConfirmed(readLh, new BookkeeperInternalCallbacks.GenericCallback<List<LedgerEntry>>() {
+ lr.forwardReadEntriesFromLastConfirmed(readLh,
+ new BookkeeperInternalCallbacks.GenericCallback<List<LedgerEntry>>() {
@Override
public void operationComplete(int rc, List<LedgerEntry> entries) {
rcHolder.set(rc);
@@ -1629,9 +1671,11 @@
lastEntryId = lastEntry.getEntryId();
}
if (lastEntryId != lh.getLastAddConfirmed()) {
- System.out.println("Inconsistent Last Add Confirmed Found for LogSegment " + metadata.getLogSegmentSequenceNumber() + ": ");
+ System.out.println("Inconsistent Last Add Confirmed Found for LogSegment "
+ + metadata.getLogSegmentSequenceNumber() + ": ");
System.out.println("\t metadata: " + metadata);
- System.out.println("\t lac in ledger metadata is " + lh.getLastAddConfirmed() + ", but lac in bookies is " + lastEntryId);
+ System.out.println("\t lac in ledger metadata is " + lh.getLastAddConfirmed()
+ + ", but lac in bookies is " + lastEntryId);
return false;
} else {
return true;
@@ -1700,7 +1744,7 @@
}
}
- static interface BKCommandRunner {
+ interface BKCommandRunner {
int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception;
}
@@ -1724,319 +1768,9 @@
return runner.run(getZooKeeperClient(), getBookKeeperClient());
}
- abstract protected int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception;
+ protected abstract int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception;
}
- /**
- static class RecoverCommand extends PerBKCommand {
-
- final List<Long> ledgers = new ArrayList<Long>();
- boolean query = false;
- boolean dryrun = false;
- boolean skipOpenLedgers = false;
- boolean fenceOnly = false;
- int fenceRate = 1;
- int concurrency = 1;
- final Set<BookieSocketAddress> bookiesSrc = new HashSet<BookieSocketAddress>();
- int partition = 0;
- int numPartitions = 0;
-
- RecoverCommand() {
- super("recover", "Recover the ledger data that stored on failed bookies");
- options.addOption("l", "ledger", true, "Specific ledger to recover");
- options.addOption("lf", "ledgerfile", true, "File contains ledgers list");
- options.addOption("q", "query", false, "Query the ledgers that contain given bookies");
- options.addOption("d", "dryrun", false, "Print the recovery plan w/o actually recovering");
- options.addOption("cy", "concurrency", true, "Number of ledgers could be recovered in parallel");
- options.addOption("sk", "skipOpenLedgers", false, "Skip recovering open ledgers");
- options.addOption("p", "partition", true, "partition");
- options.addOption("n", "num-partitions", true, "num partitions");
- options.addOption("fo", "fence-only", true, "fence the ledgers only w/o re-replicating entries");
- options.addOption("fr", "fence-rate", true, "rate on fencing ledgers");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- query = cmdline.hasOption("q");
- force = cmdline.hasOption("f");
- dryrun = cmdline.hasOption("d");
- skipOpenLedgers = cmdline.hasOption("sk");
- fenceOnly = cmdline.hasOption("fo");
- if (cmdline.hasOption("l")) {
- String[] lidStrs = cmdline.getOptionValue("l").split(",");
- try {
- for (String lidStr : lidStrs) {
- ledgers.add(Long.parseLong(lidStr));
- }
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid ledger id provided : " + cmdline.getOptionValue("l"));
- }
- }
- if (cmdline.hasOption("lf")) {
- String file = cmdline.getOptionValue("lf");
- try {
- BufferedReader br = new BufferedReader(
- new InputStreamReader(new FileInputStream(file), UTF_8.name()));
- try {
- String line = br.readLine();
-
- while (line != null) {
- ledgers.add(Long.parseLong(line));
- line = br.readLine();
- }
- } finally {
- br.close();
- }
- } catch (IOException e) {
- throw new ParseException("Invalid ledgers file provided : " + file);
- }
- }
- if (cmdline.hasOption("cy")) {
- try {
- concurrency = Integer.parseInt(cmdline.getOptionValue("cy"));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid concurrency provided : " + cmdline.getOptionValue("cy"));
- }
- }
- if (cmdline.hasOption("p")) {
- partition = Integer.parseInt(cmdline.getOptionValue("p"));
- }
- if (cmdline.hasOption("n")) {
- numPartitions = Integer.parseInt(cmdline.getOptionValue("n"));
- }
- if (cmdline.hasOption("fr")) {
- fenceRate = Integer.parseInt(cmdline.getOptionValue("fr"));
- }
- // Get bookies list to recover
- String[] args = cmdline.getArgs();
- final String[] bookieStrs = args[0].split(",");
- for (String bookieStr : bookieStrs) {
- final String bookieStrParts[] = bookieStr.split(":");
- if (bookieStrParts.length != 2) {
- throw new ParseException("BookieSrcs has invalid bookie address format (host:port expected) : "
- + bookieStr);
- }
- try {
- bookiesSrc.add(new BookieSocketAddress(bookieStrParts[0],
- Integer.parseInt(bookieStrParts[1])));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid ledger id provided : " + cmdline.getOptionValue("l"));
- }
- }
- }
-
- @Override
- protected int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
- BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
- try {
- if (query) {
- return bkQuery(bkAdmin, bookiesSrc);
- }
- if (fenceOnly) {
- return bkFence(bkc, ledgers, fenceRate);
- }
- if (!force) {
- System.out.println("Bookies : " + bookiesSrc);
- if (!IOUtils.confirmPrompt("Do you want to recover them: (Y/N)")) {
- return -1;
- }
- }
- if (!ledgers.isEmpty()) {
- System.out.println("Ledgers : " + ledgers);
- long numProcessed = 0;
- Iterator<Long> ledgersIter = ledgers.iterator();
- LinkedBlockingQueue<Long> ledgersToProcess = new LinkedBlockingQueue<Long>();
- while (ledgersIter.hasNext()) {
- long lid = ledgersIter.next();
- if (numPartitions <=0 || (numPartitions > 0 && lid % numPartitions == partition)) {
- ledgersToProcess.add(lid);
- ++numProcessed;
- }
- if (ledgersToProcess.size() == 10000) {
- System.out.println("Processing " + numProcessed + " ledgers");
- bkRecovery(ledgersToProcess, bookiesSrc, dryrun, skipOpenLedgers);
- ledgersToProcess.clear();
- System.out.println("Processed " + numProcessed + " ledgers");
- }
- }
- if (!ledgersToProcess.isEmpty()) {
- System.out.println("Processing " + numProcessed + " ledgers");
- bkRecovery(ledgersToProcess, bookiesSrc, dryrun, skipOpenLedgers);
- System.out.println("Processed " + numProcessed + " ledgers");
- }
- System.out.println("Done.");
- CountDownLatch latch = new CountDownLatch(1);
- latch.await();
- return 0;
- }
- return bkRecovery(bkAdmin, bookiesSrc, dryrun, skipOpenLedgers);
- } finally {
- bkAdmin.close();
- }
- }
-
- private int bkFence(final BookKeeperClient bkc, List<Long> ledgers, int fenceRate) throws Exception {
- if (ledgers.isEmpty()) {
- System.out.println("Nothing to fence. Done.");
- return 0;
- }
- ExecutorService executorService = Executors.newCachedThreadPool();
- final RateLimiter rateLimiter = RateLimiter.create(fenceRate);
- final byte[] passwd = getConf().getBKDigestPW().getBytes(UTF_8);
- final CountDownLatch latch = new CountDownLatch(ledgers.size());
- final AtomicInteger numPendings = new AtomicInteger(ledgers.size());
- final LinkedBlockingQueue<Long> ledgersQueue = new LinkedBlockingQueue<Long>();
- ledgersQueue.addAll(ledgers);
-
- for (int i = 0; i < concurrency; i++) {
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- while (!ledgersQueue.isEmpty()) {
- rateLimiter.acquire();
- Long lid = ledgersQueue.poll();
- if (null == lid) {
- break;
- }
- System.out.println("Fencing ledger " + lid);
- int numRetries = 3;
- while (numRetries > 0) {
- try {
- LedgerHandle lh = bkc.get().openLedger(lid, BookKeeper.DigestType.CRC32, passwd);
- lh.close();
- System.out.println("Fenced ledger " + lid + ", " + numPendings.decrementAndGet() + " left.");
- latch.countDown();
- } catch (BKException.BKNoSuchLedgerExistsException bke) {
- System.out.println("Skipped fence non-exist ledger " + lid + ", " + numPendings.decrementAndGet() + " left.");
- latch.countDown();
- } catch (BKException.BKLedgerRecoveryException lre) {
- --numRetries;
- continue;
- } catch (Exception e) {
- e.printStackTrace();
- break;
- }
- numRetries = 0;
- }
- }
- System.out.println("Thread exits");
- }
- });
- }
- latch.await();
- SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
- return 0;
- }
-
- private int bkQuery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddrs)
- throws InterruptedException, BKException {
- SortedMap<Long, LedgerMetadata> ledgersContainBookies =
- bkAdmin.getLedgersContainBookies(bookieAddrs);
- System.err.println("NOTE: Bookies in inspection list are marked with '*'.");
- for (Map.Entry<Long, LedgerMetadata> ledger : ledgersContainBookies.entrySet()) {
- System.out.println("ledger " + ledger.getKey() + " : " + ledger.getValue().getState());
- Map<Long, Integer> numBookiesToReplacePerEnsemble =
- inspectLedger(ledger.getValue(), bookieAddrs);
- System.out.print("summary: [");
- for (Map.Entry<Long, Integer> entry : numBookiesToReplacePerEnsemble.entrySet()) {
- System.out.print(entry.getKey() + "=" + entry.getValue() + ", ");
- }
- System.out.println("]");
- System.out.println();
- }
- System.out.println("Done");
- return 0;
- }
-
- private Map<Long, Integer> inspectLedger(LedgerMetadata metadata, Set<BookieSocketAddress> bookiesToInspect) {
- Map<Long, Integer> numBookiesToReplacePerEnsemble = new TreeMap<Long, Integer>();
- for (Map.Entry<Long, ArrayList<BookieSocketAddress>> ensemble : metadata.getEnsembles().entrySet()) {
- ArrayList<BookieSocketAddress> bookieList = ensemble.getValue();
- System.out.print(ensemble.getKey() + ":\t");
- int numBookiesToReplace = 0;
- for (BookieSocketAddress bookie: bookieList) {
- System.out.print(bookie.toString());
- if (bookiesToInspect.contains(bookie)) {
- System.out.print("*");
- ++numBookiesToReplace;
- } else {
- System.out.print(" ");
- }
- System.out.print(" ");
- }
- System.out.println();
- numBookiesToReplacePerEnsemble.put(ensemble.getKey(), numBookiesToReplace);
- }
- return numBookiesToReplacePerEnsemble;
- }
-
- private int bkRecovery(final LinkedBlockingQueue<Long> ledgers, final Set<BookieSocketAddress> bookieAddrs,
- final boolean dryrun, final boolean skipOpenLedgers)
- throws Exception {
- return runBKCommand(new BKCommandRunner() {
- @Override
- public int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
- BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
- try {
- bkRecovery(bkAdmin, ledgers, bookieAddrs, dryrun, skipOpenLedgers);
- return 0;
- } finally {
- bkAdmin.close();
- }
- }
- });
- }
-
- private int bkRecovery(final BookKeeperAdmin bkAdmin, final LinkedBlockingQueue<Long> ledgers,
- final Set<BookieSocketAddress> bookieAddrs,
- final boolean dryrun, final boolean skipOpenLedgers)
- throws InterruptedException, BKException {
- final AtomicInteger numPendings = new AtomicInteger(ledgers.size());
- final ExecutorService executorService = Executors.newCachedThreadPool();
- final CountDownLatch doneLatch = new CountDownLatch(concurrency);
- Runnable r = new Runnable() {
- @Override
- public void run() {
- while (!ledgers.isEmpty()) {
- long lid = -1L;
- try {
- lid = ledgers.take();
- System.out.println("Recovering ledger " + lid);
- bkAdmin.recoverBookieData(lid, bookieAddrs, dryrun, skipOpenLedgers);
- System.out.println("Recovered ledger completed : " + lid + ", " + numPendings.decrementAndGet() + " left");
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- doneLatch.countDown();
- break;
- } catch (BKException ke) {
- System.out.println("Recovered ledger failed : " + lid + ", rc = " + BKException.getMessage(ke.getCode()));
- }
- }
- doneLatch.countDown();
- }
- };
- for (int i = 0; i < concurrency; i++) {
- executorService.submit(r);
- }
- doneLatch.await();
- SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
- return 0;
- }
-
- private int bkRecovery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddrs,
- boolean dryrun, boolean skipOpenLedgers)
- throws InterruptedException, BKException {
- bkAdmin.recoverBookieData(bookieAddrs, dryrun, skipOpenLedgers);
- return 0;
- }
-
- @Override
- protected String getUsage() {
- return "recover [options] <bookiesSrc>";
- }
- }
- **/
/**
* Per Ledger Command, which parse common options for per ledger. e.g. ledger id.
@@ -2068,6 +1802,9 @@
}
}
+ /**
+ * Command used to force recover ledger.
+ */
protected static class RecoverLedgerCommand extends PerLedgerCommand {
RecoverLedgerCommand() {
@@ -2106,6 +1843,9 @@
}
}
+ /**
+ * Command used to find the stream for a given ledger.
+ */
protected static class FindLedgerCommand extends PerLedgerCommand {
FindLedgerCommand() {
@@ -2142,6 +1882,9 @@
}
}
+ /**
+ * Command used to read last add confirmed for a given ledger.
+ */
protected static class ReadLastConfirmedCommand extends PerLedgerCommand {
ReadLastConfirmedCommand() {
@@ -2167,6 +1910,9 @@
}
}
+ /**
+ * Command used to read entries for a given ledger.
+ */
protected static class ReadEntriesCommand extends PerLedgerCommand {
Long fromEntryId;
@@ -2186,7 +1932,8 @@
options.addOption("fid", "from", true, "Entry id to start reading");
options.addOption("uid", "until", true, "Entry id to read until");
options.addOption("bks", "all-bookies", false, "Read entry from all bookies");
- options.addOption("lac", "last-add-confirmed", false, "Return last add confirmed rather than entry payload");
+ options.addOption("lac", "last-add-confirmed", false,
+ "Return last add confirmed rather than entry payload");
options.addOption("ver", "metadata-version", true, "The log segment metadata version to use");
options.addOption("bad", "corrupt-only", false, "Display info for corrupt entries only");
}
@@ -2212,8 +1959,9 @@
@Override
protected int runCmd() throws Exception {
- LedgerHandle lh = getBookKeeperClient().get().openLedgerNoRecovery(getLedgerID(), BookKeeper.DigestType.CRC32,
- dlConf.getBKDigestPW().getBytes(UTF_8));
+ LedgerHandle lh = getBookKeeperClient().get()
+ .openLedgerNoRecovery(getLedgerID(), BookKeeper.DigestType.CRC32,
+ dlConf.getBKDigestPW().getBytes(UTF_8));
try {
if (null == fromEntryId) {
fromEntryId = 0L;
@@ -2241,7 +1989,8 @@
return 0;
}
- private void readEntriesFromAllBookies(LedgerReader ledgerReader, LedgerHandle lh, long fromEntryId, long untilEntryId)
+ private void readEntriesFromAllBookies(LedgerReader ledgerReader,
+ LedgerHandle lh, long fromEntryId, long untilEntryId)
throws Exception {
for (long eid = fromEntryId; eid <= untilEntryId; ++eid) {
final CountDownLatch doneLatch = new CountDownLatch(1);
@@ -2297,13 +2046,15 @@
}
}
- private void readLacsFromAllBookies(LedgerReader ledgerReader, LedgerHandle lh, long fromEntryId, long untilEntryId)
+ private void readLacsFromAllBookies(LedgerReader ledgerReader,
+ LedgerHandle lh, long fromEntryId, long untilEntryId)
throws Exception {
for (long eid = fromEntryId; eid <= untilEntryId; ++eid) {
final CountDownLatch doneLatch = new CountDownLatch(1);
final AtomicReference<Set<LedgerReader.ReadResult<Long>>> resultHolder =
new AtomicReference<Set<LedgerReader.ReadResult<Long>>>();
- ledgerReader.readLacs(lh, eid, new BookkeeperInternalCallbacks.GenericCallback<Set<LedgerReader.ReadResult<Long>>>() {
+ ledgerReader.readLacs(lh, eid,
+ new BookkeeperInternalCallbacks.GenericCallback<Set<LedgerReader.ReadResult<Long>>>() {
@Override
public void operationComplete(int rc, Set<LedgerReader.ReadResult<Long>> readResults) {
if (BKException.Code.OK == rc) {
@@ -2374,7 +2125,10 @@
}
}
- protected static abstract class AuditCommand extends OptsCommand {
+ /**
+ * Command associated with audit.
+ */
+ protected abstract static class AuditCommand extends OptsCommand {
protected final Options options = new Options();
protected final DistributedLogConfiguration dlConf;
@@ -2477,7 +2231,7 @@
}
if (cmdline.hasOption("ap")) {
String[] aps = cmdline.getOptionValue("ap").split(",");
- for(String ap : aps) {
+ for (String ap : aps) {
List<String> list = new ArrayList<String>();
String[] array = ap.split(";");
Collections.addAll(list, array);
@@ -2526,6 +2280,9 @@
}
}
+ /**
+ * Audit stream space usage for a given dl uri.
+ */
public static class AuditDLSpaceCommand extends PerDLCommand {
private String regex = null;
@@ -2590,6 +2347,9 @@
}
}
+ /**
+ * Audit bk space usage for a given dl uri.
+ */
public static class AuditBKSpaceCommand extends PerDLCommand {
AuditBKSpaceCommand() {
@@ -2614,6 +2374,9 @@
}
}
+ /**
+ * Command used to truncate a stream at a specific position.
+ */
protected static class TruncateStreamCommand extends PerStreamCommand {
DLSN dlsn = DLSN.InvalidDLSN;
@@ -2647,7 +2410,8 @@
long totalRecords = dlm.getLogRecordCount();
long recordsAfterTruncate = FutureUtils.result(dlm.getLogRecordCountAsync(dlsn));
long recordsToTruncate = totalRecords - recordsAfterTruncate;
- if (!getForce() && !IOUtils.confirmPrompt("Do you want to truncate " + streamName + " at dlsn " + dlsn + " (" + recordsToTruncate + " records)?")) {
+ if (!getForce() && !IOUtils.confirmPrompt("Do you want to truncate "
+ + streamName + " at dlsn " + dlsn + " (" + recordsToTruncate + " records)?")) {
return 0;
} else {
AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
@@ -2669,6 +2433,9 @@
}
}
+ /**
+ * Command used to Deserialize DLSN.
+ */
public static class DeserializeDLSNCommand extends SimpleCommand {
String base64Dlsn = "";
@@ -2693,6 +2460,9 @@
}
}
+ /**
+ * Command used to Serialize DLSN.
+ */
public static class SerializeDLSNCommand extends SimpleCommand {
private DLSN dlsn = DLSN.InitialDLSN;
@@ -2724,6 +2494,9 @@
}
}
+ /**
+ * Command used to Delete the subscriber in subscription store.
+ */
public static class DeleteSubscriberCommand extends PerDLCommand {
int numThreads = 1;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/Tool.java b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/Tool.java
index 9045fa2..bc5603c 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/Tool.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/Tool.java
@@ -17,6 +17,8 @@
*/
package org.apache.distributedlog.tools;
+import java.util.Map;
+import java.util.TreeMap;
import org.apache.bookkeeper.util.ReflectionUtils;
import org.apache.commons.cli.BasicParser;
import org.apache.commons.cli.CommandLine;
@@ -24,11 +26,10 @@
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
-import java.util.Map;
-import java.util.TreeMap;
+
/**
- * A Tool Framework
+ * A Tool Framework.
*/
public abstract class Tool {
@@ -106,7 +107,9 @@
helpFormatter.printHelp(getUsage(), getOptions());
}
}
-
+ /**
+ * HelpCommand describing the usage of this tool.
+ */
public class HelpCommand implements Command {
@Override
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/package-info.java
index b410188..0b79fc7 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * Tools for distributedlog
+ * Tools for distributedlog.
*/
package org.apache.distributedlog.tools;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Allocator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Allocator.java
index a9b81e2..91403ff 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Allocator.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Allocator.java
@@ -24,14 +24,14 @@
import org.apache.distributedlog.util.Transaction.OpListener;
/**
- * A common interface to allocate <i>I</i> under transaction <i>T</i>.
+ * A common interface to allocate <i>R</i> under transaction <i>T</i>.
*
* <h3>Usage Example</h3>
*
- * Here is an example on demonstrating how `Allocator` works.
+ *<p> Here is an example on demonstrating how `Allocator` works.</p>
*
* <pre> {@code
- * Allocator<I, T, R> allocator = ...;
+ * Allocator<R, T> allocator = ...;
*
* // issue an allocate request
* try {
@@ -45,16 +45,16 @@
* // Start a transaction
* final Transaction<T> txn = ...;
*
- * // Try obtain object I
- * CompletableFuture<I> tryObtainFuture = allocator.tryObtain(txn, new OpListener<I>() {
- * public void onCommit(I resource) {
+ * // Try obtain object R
+ * CompletableFuture<R> tryObtainFuture = allocator.tryObtain(txn, new OpListener<R>() {
+ * public void onCommit(R resource) {
* // the obtain succeed, process with the resource
* }
* public void onAbort() {
* // the obtain failed.
* }
* }).addFutureEventListener(new FutureEventListener() {
- * public void onSuccess(I resource) {
+ * public void onSuccess(R resource) {
* // the try obtain succeed. but the obtain has not been confirmed or aborted.
* // execute the transaction to confirm if it could complete obtain
* txn.execute();
@@ -66,36 +66,35 @@
*
* }</pre>
*/
-public interface Allocator<I, T> extends AsyncCloseable, AsyncDeleteable {
+public interface Allocator<R, T> extends AsyncCloseable, AsyncDeleteable {
/**
- * Issue allocation request to allocate <i>I</i>.
+ * Issue allocation request to allocate <i>R</i>.
* The implementation should be non-blocking call.
*
* @throws IOException
- * if fail to request allocating a <i>I</i>.
+ * if fail to request allocating a <i>R</i>.
*/
void allocate() throws IOException;
/**
- * Try obtaining an <i>I</i> in a given transaction <i>T</i>. The object obtained is tentative.
+ * Try obtaining an <i>R</i> in a given transaction <i>T</i>. The object obtained is tentative.
* Whether the object is obtained or aborted is determined by the result of the execution. You could
* register a listener under this `tryObtain` operation to know whether the object is obtained or
* aborted.
*
- * <p>
- * It is a typical two-phases operation on obtaining a resource from allocator.
+ * <p>It is a typical two-phases operation on obtaining a resource from allocator.
* The future returned by this method acts as a `prepare` operation, the resource is tentative obtained
* from the allocator. The execution of the txn acts as a `commit` operation, the resource is confirmed
* to be obtained by this transaction. <code>listener</code> is for the whole completion of the obtain.
- * <p>
- * <code>listener</code> is only triggered after `prepare` succeed. if `prepare` failed, no actions will
+ *
+ * <p><code>listener</code> is only triggered after `prepare` succeed. if `prepare` failed, no actions will
* happen to the listener.
*
* @param txn
* transaction.
- * @return future result returning <i>I</i> that would be obtained under transaction <code>txn</code>.
+ * @return future result returning <i>R</i> that would be obtained under transaction <code>txn</code>.
*/
- CompletableFuture<I> tryObtain(Transaction<T> txn, OpListener<I> listener);
+ CompletableFuture<R> tryObtain(Transaction<T> txn, OpListener<R> listener);
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java
index 5113e54..a999f71 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java
@@ -21,7 +21,7 @@
import org.apache.commons.cli.CommandLine;
/**
- * Utils to commandline
+ * Utils to commandline.
*/
public class CommandLineUtils {
@@ -41,7 +41,8 @@
}
}
- public static Optional<Integer> getOptionalIntegerArg(CommandLine cmdline, String arg) throws IllegalArgumentException {
+ public static Optional<Integer> getOptionalIntegerArg(CommandLine cmdline, String arg)
+ throws IllegalArgumentException {
try {
if (cmdline.hasOption(arg)) {
return Optional.of(Integer.parseInt(cmdline.getOptionValue(arg)));
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/ConfUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/ConfUtils.java
index aa7cefe..8c51f9e 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/ConfUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/ConfUtils.java
@@ -17,13 +17,16 @@
*/
package org.apache.distributedlog.util;
+import java.util.Iterator;
+import org.apache.commons.configuration.Configuration;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.commons.configuration.Configuration;
-import java.util.Iterator;
+/**
+ * Utils to conf.
+ */
public class ConfUtils {
/**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/DLUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/DLUtils.java
index 7231105..833c9f3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/DLUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/DLUtils.java
@@ -17,21 +17,23 @@
*/
package org.apache.distributedlog.util;
+import static com.google.common.base.Charsets.UTF_8;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
import com.google.common.base.Objects;
-import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.exceptions.InvalidStreamNameException;
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.commons.lang.StringUtils;
-
import java.net.InetAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.List;
+import org.apache.commons.lang.StringUtils;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
-import static com.google.common.base.Charsets.UTF_8;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
+
+
+
/**
* Utilities about DL implementations like uri, log segments, metadata serialization and deserialization.
@@ -249,7 +251,7 @@
private static String getHostIpLockClientId() {
try {
return InetAddress.getLocalHost().toString();
- } catch(Exception ex) {
+ } catch (Exception ex) {
return DistributedLogConstants.UNKNOWN_CLIENT_ID;
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java
index 8b5cc65..d37e865 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java
@@ -23,9 +23,15 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+/**
+ * Utils to Failpoint.
+ */
public class FailpointUtils {
- static final Logger logger = LoggerFactory.getLogger(FailpointUtils.class);
+ private static final Logger logger = LoggerFactory.getLogger(FailpointUtils.class);
+ /**
+ * enum for FailPointName.
+ */
public enum FailPointName {
FP_StartLogSegmentBeforeLedgerCreate,
FP_StartLogSegmentAfterLedgerCreate,
@@ -44,12 +50,18 @@
FP_LogWriterIssuePending,
}
- public static interface FailPointAction {
+ /**
+ * interface for FailPointAction.
+ */
+ public interface FailPointAction {
boolean checkFailPoint() throws IOException;
boolean checkFailPointNoThrow();
}
- public static abstract class AbstractFailPointAction implements FailPointAction {
+ /**
+ * AbstractFailPointAction.
+ */
+ public abstract static class AbstractFailPointAction implements FailPointAction {
@Override
public boolean checkFailPointNoThrow() {
try {
@@ -74,7 +86,9 @@
throw new IOException("Throw ioexception for failure point");
}
};
-
+ /**
+ * enum for fail point actions.
+ */
public enum FailPointActions {
FailPointAction_Default,
FailPointAction_Throw
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/RetryPolicyUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/RetryPolicyUtils.java
index 87d4d61..0473ef3 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/RetryPolicyUtils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/RetryPolicyUtils.java
@@ -21,12 +21,12 @@
import org.apache.bookkeeper.zookeeper.RetryPolicy;
/**
- * Utils for {@link org.apache.bookkeeper.zookeeper.RetryPolicy}
+ * Utils for {@link org.apache.bookkeeper.zookeeper.RetryPolicy}.
*/
public class RetryPolicyUtils {
/**
- * Infinite retry policy
+ * Infinite retry policy.
*/
public static final RetryPolicy DEFAULT_INFINITE_RETRY_POLICY = infiniteRetry(200, 2000);
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java
index 35cdfa0..c54de93 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.java
@@ -21,10 +21,11 @@
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.stats.Counter;
import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
import org.apache.distributedlog.common.util.PermitLimiter;
/**
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Transaction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Transaction.java
index d90a7f8..f3a75eb 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Transaction.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Transaction.java
@@ -21,34 +21,34 @@
import java.util.concurrent.CompletableFuture;
/**
- * Util class represents a transaction
+ * Util class represents a transaction.
*/
@Beta
-public interface Transaction<OpResult> {
+public interface Transaction<OpResultT> {
/**
* An operation executed in a transaction.
*/
- interface Op<OpResult> {
+ interface Op<OpResultT> {
/**
- * Execute after the transaction succeeds
+ * Execute after the transaction succeeds.
*/
- void commit(OpResult r);
+ void commit(OpResultT r);
/**
- * Execute after the transaction fails
+ * Execute after the transaction fails.
*/
- void abort(Throwable t, OpResult r);
+ void abort(Throwable t, OpResultT r);
}
/**
* Listener on the result of an {@link Transaction.Op}.
*
- * @param <OpResult>
+ * @param <OpResultT>
*/
- interface OpListener<OpResult> {
+ interface OpListener<OpResultT> {
/**
* Trigger on operation committed.
@@ -56,7 +56,7 @@
* @param r
* result to return
*/
- void onCommit(OpResult r);
+ void onCommit(OpResultT r);
/**
* Trigger on operation aborted.
@@ -73,7 +73,7 @@
* @param operation
* operation to execute under current transaction
*/
- void addOp(Op<OpResult> operation);
+ void addOp(Op<OpResultT> operation);
/**
* Execute the current transaction. If the transaction succeed, all operations will be
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Utils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Utils.java
index 99a4155..76f7978 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Utils.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Utils.java
@@ -17,6 +17,10 @@
*/
package org.apache.distributedlog.util;
+import com.google.common.base.Objects;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
@@ -28,28 +32,24 @@
import java.util.concurrent.atomic.AtomicInteger;
import javax.annotation.Nullable;
-import com.google.common.base.Objects;
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.google.common.io.Closeables;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Versioned;
import org.apache.distributedlog.DistributedLogConstants;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.functions.VoidFunctions;
import org.apache.distributedlog.exceptions.BKTransmitException;
import org.apache.distributedlog.exceptions.DLInterruptedException;
import org.apache.distributedlog.exceptions.UnexpectedException;
import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.common.functions.VoidFunctions;
import org.apache.distributedlog.io.AsyncAbortable;
import org.apache.distributedlog.io.AsyncCloseable;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
@@ -72,8 +72,7 @@
}
/**
- * Current time from some fixed base time - so useful for cross machine
- * comparison
+ * Current time from some fixed base time - so useful for cross machine comparison.
*
* @return current time in milliseconds.
*/
@@ -83,7 +82,7 @@
/**
* Milliseconds elapsed since the time specified, the input is nanoTime
- * the only conversion happens when computing the elapsed time
+ * the only conversion happens when computing the elapsed time.
*
* @param startMsecTime the start of the interval that we are measuring
* @return elapsed time in milliseconds.
@@ -170,7 +169,8 @@
return;
}
String parent = pathToCreate.substring(0, lastSlash);
- if (parentPathShouldNotCreate.isPresent() && Objects.equal(parentPathShouldNotCreate.get(), parent)) {
+ if (parentPathShouldNotCreate.isPresent()
+ && Objects.equal(parentPathShouldNotCreate.get(), parent)) {
// we should stop here
callback.processResult(rc, path, ctx, name);
return;
@@ -179,10 +179,11 @@
CreateMode.PERSISTENT, new AsyncCallback.StringCallback() {
@Override
public void processResult(int rc, String path, Object ctx, String name) {
- if (rc == KeeperException.Code.OK.intValue() || rc == KeeperException.Code.NODEEXISTS.intValue()) {
+ if (rc == KeeperException.Code.OK.intValue()
+ || rc == KeeperException.Code.NODEEXISTS.intValue()) {
// succeeded in creating the parent, now create the original path
- zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
- data, acl, createMode, callback, ctx);
+ zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate,
+ parentPathShouldNotCreate, data, acl, createMode, callback, ctx);
} else {
callback.processResult(rc, path, ctx, name);
}
@@ -191,9 +192,11 @@
}
}, ctx);
} catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
- callback.processResult(DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE, zkce.getMessage(), ctx, pathToCreate);
+ callback.processResult(DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE,
+ zkce.getMessage(), ctx, pathToCreate);
} catch (InterruptedException ie) {
- callback.processResult(DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE, ie.getMessage(), ctx, pathToCreate);
+ callback.processResult(DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE,
+ ie.getMessage(), ctx, pathToCreate);
}
}
@@ -223,7 +226,7 @@
}
/**
- * Asynchronously create zookeeper path recursively and optimistically
+ * Asynchronously create zookeeper path recursively and optimistically.
*
* @param zkc Zookeeper client
* @param pathToCreate Zookeeper full path
@@ -349,7 +352,8 @@
return promise;
}
- public static CompletableFuture<ZkVersion> zkSetData(ZooKeeperClient zkc, String path, byte[] data, ZkVersion version) {
+ public static CompletableFuture<ZkVersion> zkSetData(ZooKeeperClient zkc,
+ String path, byte[] data, ZkVersion version) {
ZooKeeper zk;
try {
zk = zkc.get();
@@ -457,7 +461,7 @@
zk.delete(path, version.getZnodeVersion(), new AsyncCallback.VoidCallback() {
@Override
public void processResult(int rc, String path, Object ctx) {
- if (KeeperException.Code.OK.intValue() == rc ) {
+ if (KeeperException.Code.OK.intValue() == rc) {
promise.complete(true);
} else if (KeeperException.Code.NONODE.intValue() == rc) {
promise.complete(false);
@@ -719,7 +723,7 @@
}
/**
- * Abort async <i>abortable</i>
+ * Abort async <i>abortable</i>.
*
* @param abortable the {@code AsyncAbortable} object to be aborted, or null, in which case this method
* does nothing.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/package-info.java
index 7371ef4..4a99666 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/util/package-info.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/package-info.java
@@ -16,6 +16,6 @@
* limitations under the License.
*/
/**
- * DistributedLog Utils
+ * DistributedLog Utils.
*/
package org.apache.distributedlog.util;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/DefaultZKOp.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/DefaultZKOp.java
index 45120d4..4ade77f 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/DefaultZKOp.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/DefaultZKOp.java
@@ -17,11 +17,11 @@
*/
package org.apache.distributedlog.zk;
+import javax.annotation.Nullable;
import org.apache.distributedlog.util.Transaction.OpListener;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
-import javax.annotation.Nullable;
/**
* Default zookeeper operation. No action on commiting or aborting.
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java
index 1dd702f..bd731a6 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java
@@ -17,20 +17,21 @@
*/
package org.apache.distributedlog.zk;
-import org.apache.distributedlog.common.util.PermitManager;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.common.util.PermitManager;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
/**
* Manager to control all the log segments rolling.
@@ -39,7 +40,7 @@
static final Logger LOG = LoggerFactory.getLogger(LimitedPermitManager.class);
- static enum PermitState {
+ enum PermitState {
ALLOWED, DISALLOWED, DISABLED
}
@@ -103,7 +104,7 @@
}
@Override
- synchronized public Permit acquirePermit() {
+ public synchronized Permit acquirePermit() {
if (!enablePermits) {
return new EpochPermit(PermitState.DISABLED);
}
@@ -116,7 +117,7 @@
}
@Override
- synchronized public void releasePermit(Permit permit) {
+ public synchronized void releasePermit(Permit permit) {
if (null != semaphore && permit.isAllowed()) {
if (period <= 0) {
semaphore.release();
@@ -124,8 +125,8 @@
try {
executorService.schedule(this, period, timeUnit);
} catch (RejectedExecutionException ree) {
- LOG.warn("Failed on scheduling releasing permit in given period ({}ms)." +
- " Release it immediately : ", timeUnit.toMillis(period), ree);
+ LOG.warn("Failed on scheduling releasing permit in given period ({}ms)."
+ + " Release it immediately : ", timeUnit.toMillis(period), ree);
semaphore.release();
}
}
@@ -133,11 +134,11 @@
}
@Override
- synchronized public boolean disallowObtainPermits(Permit permit) {
+ public synchronized boolean disallowObtainPermits(Permit permit) {
if (!(permit instanceof EpochPermit)) {
return false;
}
- if (epoch.getAndIncrement() == ((EpochPermit)permit).getEpoch()) {
+ if (epoch.getAndIncrement() == ((EpochPermit) permit).getEpoch()) {
this.enablePermits = false;
LOG.info("EnablePermits = {}, Epoch = {}.", this.enablePermits, epoch.get());
return true;
@@ -152,7 +153,7 @@
}
@Override
- synchronized public boolean allowObtainPermits() {
+ public synchronized boolean allowObtainPermits() {
forceSetAllowPermits(true);
return true;
}
@@ -188,7 +189,7 @@
}
public void unregisterGauge() {
- if(this.statsLogger != null && this.outstandingGauge != null) {
+ if (this.statsLogger != null && this.outstandingGauge != null) {
this.statsLogger.scope("permits").unregisterGauge("outstanding", this.outstandingGauge);
}
}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKOp.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKOp.java
index 39e4c30..715c5ec 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKOp.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKOp.java
@@ -17,14 +17,15 @@
*/
package org.apache.distributedlog.zk;
+import javax.annotation.Nullable;
import org.apache.distributedlog.util.Transaction;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
-import javax.annotation.Nullable;
+
/**
- * ZooKeeper Transaction Operation
+ * ZooKeeper Transaction Operation.
*/
public abstract class ZKOp implements Transaction.Op<Object> {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
index aeabbfa..3502906 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
@@ -30,7 +30,7 @@
import org.apache.zookeeper.OpResult;
/**
- * ZooKeeper Transaction
+ * ZooKeeper Transaction.
*/
public class ZKTransaction implements Transaction<Object>, AsyncCallback.MultiCallback {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKVersionedSetOp.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKVersionedSetOp.java
index 6ab5973..947b112 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKVersionedSetOp.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKVersionedSetOp.java
@@ -17,17 +17,17 @@
*/
package org.apache.distributedlog.zk;
-import org.apache.distributedlog.util.Transaction.OpListener;
+import javax.annotation.Nullable;
import org.apache.bookkeeper.meta.ZkVersion;
import org.apache.bookkeeper.versioning.Version;
+import org.apache.distributedlog.util.Transaction.OpListener;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
-import javax.annotation.Nullable;
/**
- * ZooKeeper Operation that plays with {@link org.apache.bookkeeper.versioning.Version}
+ * ZooKeeper Operation that plays with {@link org.apache.bookkeeper.versioning.Version}.
*/
public class ZKVersionedSetOp extends ZKOp {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java
index 670b5d2..759cf79 100644
--- a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java
@@ -17,9 +17,14 @@
*/
package org.apache.distributedlog.zk;
-import org.apache.distributedlog.ZooKeeperClient;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.bookkeeper.stats.Gauge;
import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.distributedlog.ZooKeeperClient;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
@@ -27,11 +32,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
+
/**
* Watcher Manager to manage watchers.
@@ -43,35 +44,38 @@
*/
public class ZKWatcherManager implements Watcher {
- static final Logger logger = LoggerFactory.getLogger(ZKWatcherManager.class);
+ private static final Logger logger = LoggerFactory.getLogger(ZKWatcherManager.class);
public static Builder newBuilder() {
return new Builder();
}
+ /**
+ * Builder to build Watcher Manager.
+ */
public static class Builder {
- private String _name;
- private StatsLogger _statsLogger;
- private ZooKeeperClient _zkc;
+ private String pName;
+ private StatsLogger pStatsLogger;
+ private ZooKeeperClient pZkc;
public Builder name(String name) {
- this._name = name;
+ this.pName = name;
return this;
}
public Builder zkc(ZooKeeperClient zkc) {
- this._zkc = zkc;
+ this.pZkc = zkc;
return this;
}
public Builder statsLogger(StatsLogger statsLogger) {
- this._statsLogger = statsLogger;
+ this.pStatsLogger = statsLogger;
return this;
}
public ZKWatcherManager build() {
- return new ZKWatcherManager(_name, _zkc, _statsLogger);
+ return new ZKWatcherManager(pName, pZkc, pStatsLogger);
}
}
@@ -164,7 +168,8 @@
// best-efforts to remove watches
try {
if (null != zkc && removeFromServer) {
- zkc.get().removeWatches(path, this, WatcherType.Children, true, new AsyncCallback.VoidCallback() {
+ zkc.get().removeWatches(path, this, WatcherType.Children, true,
+ new AsyncCallback.VoidCallback() {
@Override
public void processResult(int rc, String path, Object ctx) {
if (KeeperException.Code.OK.intValue() == rc) {
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/package-info.java
new file mode 100644
index 0000000..98f238b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * zk utils for distributedlog.
+ */
+package org.apache.distributedlog.zk;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java b/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
index d821b05..7e1464c 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
@@ -17,29 +17,12 @@
*/
package org.apache.distributedlog;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
-import org.apache.distributedlog.api.AsyncLogWriter;
-import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.api.LogReader;
-import org.apache.distributedlog.api.MetadataAccessor;
-import org.apache.distributedlog.api.namespace.Namespace;
-import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
-import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.common.util.PermitLimiter;
-import org.apache.distributedlog.util.Utils;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.versioning.Version;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.IOException;
import java.net.URI;
import java.net.URL;
@@ -48,31 +31,50 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.feature.SettableFeatureProvider;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.api.MetadataAccessor;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.util.PermitLimiter;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.util.ConfUtils;
+import org.apache.distributedlog.util.Utils;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertNotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
/**
* Utility class for setting up bookkeeper ensembles
- * and bringing individual bookies up and down
+ * and bringing individual bookies up and down.
*/
public class DLMTestUtil {
protected static final Logger LOG = LoggerFactory.getLogger(DLMTestUtil.class);
- private final static byte[] payloadStatic = repeatString("abc", 512).getBytes();
+ private static final byte[] payloadStatic = repeatString("abc", 512).getBytes();
static String repeatString(String s, int n) {
String ret = s;
- for(int i = 1; i < n; i++) {
+ for (int i = 1; i < n; i++) {
ret += s;
}
return ret;
}
- public static Map<Long, LogSegmentMetadata> readLogSegments(ZooKeeperClient zkc, String ledgerPath) throws Exception {
+ public static Map<Long, LogSegmentMetadata> readLogSegments(ZooKeeperClient zkc, String ledgerPath)
+ throws Exception {
List<String> children = zkc.get().getChildren(ledgerPath, false);
LOG.info("Children under {} : {}", ledgerPath, children);
Map<Long, LogSegmentMetadata> segments =
@@ -113,7 +115,8 @@
try {
List<LogSegmentMetadata> logSegmentList = dlm.getLogSegments();
LogSegmentMetadata lastSegment = logSegmentList.get(logSegmentList.size() - 1);
- LogSegmentEntryStore entryStore = dlm.getNamespaceDriver().getLogSegmentEntryStore(NamespaceDriver.Role.READER);
+ LogSegmentEntryStore entryStore =
+ dlm.getNamespaceDriver().getLogSegmentEntryStore(NamespaceDriver.Role.READER);
Utils.close(Utils.ioResult(entryStore.openRandomAccessReader(lastSegment, true)));
} finally {
dlm.close();
@@ -208,7 +211,7 @@
public static LogRecordWithDLSN getLogRecordWithDLSNInstance(DLSN dlsn, long txId, boolean isControlRecord) {
LogRecordWithDLSN record = new LogRecordWithDLSN(dlsn, txId, generatePayload(txId), 1L);
- record.setPositionWithinLogSegment((int)txId + 1);
+ record.setPositionWithinLogSegment((int) txId + 1);
if (isControlRecord) {
record.setControl();
}
@@ -219,14 +222,16 @@
return String.format("%s_%018d", DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX, logSegmentSeqNo);
}
- public static String completedLedgerZNodeNameWithVersion(long ledgerId, long firstTxId, long lastTxId, long logSegmentSeqNo) {
+ public static String completedLedgerZNodeNameWithVersion(long ledgerId,
+ long firstTxId, long lastTxId, long logSegmentSeqNo) {
return String.format("%s_%018d_%018d_%018d_v%dl%d_%04d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX,
- firstTxId, lastTxId, logSegmentSeqNo, DistributedLogConstants.LOGSEGMENT_NAME_VERSION, ledgerId,
- DistributedLogConstants.LOCAL_REGION_ID);
+ firstTxId, lastTxId, logSegmentSeqNo, DistributedLogConstants.LOGSEGMENT_NAME_VERSION, ledgerId,
+ DistributedLogConstants.LOCAL_REGION_ID);
}
public static String completedLedgerZNodeNameWithTxID(long firstTxId, long lastTxId) {
- return String.format("%s_%018d_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, firstTxId, lastTxId);
+ return String.format("%s_%018d_%018d",
+ DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, firstTxId, lastTxId);
}
public static String completedLedgerZNodeNameWithLogSegmentSequenceNumber(long logSegmentSeqNo) {
@@ -286,7 +291,8 @@
.setInprogress(false)
.setLogSegmentSequenceNo(logSegmentSeqNo)
.build();
- return metadata.completeLogSegment(ledgerPath + "/" + completedLedgerZNodeNameWithLogSegmentSequenceNumber(logSegmentSeqNo),
+ return metadata.completeLogSegment(ledgerPath + "/"
+ + completedLedgerZNodeNameWithLogSegmentSequenceNumber(logSegmentSeqNo),
lastTxId, recordCount, lastEntryId, lastSlotId, firstTxId);
}
@@ -303,12 +309,13 @@
}
}
- public static long generateLogSegmentNonPartitioned(DistributedLogManager dlm, int controlEntries, int userEntries, long startTxid)
- throws Exception {
+ public static long generateLogSegmentNonPartitioned(DistributedLogManager dlm,
+ int controlEntries, int userEntries, long startTxid) throws Exception {
return generateLogSegmentNonPartitioned(dlm, controlEntries, userEntries, startTxid, 1L);
}
- public static long generateLogSegmentNonPartitioned(DistributedLogManager dlm, int controlEntries, int userEntries, long startTxid, long txidStep) throws Exception {
+ public static long generateLogSegmentNonPartitioned(DistributedLogManager dlm, int controlEntries,
+ int userEntries, long startTxid, long txidStep) throws Exception {
AsyncLogWriter out = dlm.startAsyncLogSegmentNonPartitioned();
long txid = startTxid;
for (int i = 0; i < controlEntries; ++i) {
@@ -334,9 +341,9 @@
return ((BKNamespaceDriver) dlm.getNamespaceDriver()).getReaderBKC();
}
- public static void injectLogSegmentWithGivenLogSegmentSeqNo(DistributedLogManager manager, DistributedLogConfiguration conf,
- long logSegmentSeqNo, long startTxID, boolean writeEntries, long segmentSize,
- boolean completeLogSegment)
+ public static void injectLogSegmentWithGivenLogSegmentSeqNo(DistributedLogManager manager,
+ DistributedLogConfiguration conf, long logSegmentSeqNo, long startTxID,
+ boolean writeEntries, long segmentSize, boolean completeLogSegment)
throws Exception {
BKDistributedLogManager dlm = (BKDistributedLogManager) manager;
BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
@@ -435,7 +442,8 @@
if (recordWrongLastDLSN) {
Utils.ioResult(writer.asyncClose());
writeHandler.completeAndCloseLogSegment(
- writeHandler.inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
+ writeHandler.inprogressZNodeName(writer.getLogSegmentId(),
+ writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
writer.getLogSegmentSequenceNumber(),
writer.getLogSegmentId(),
writer.getStartTxId(),
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java b/distributedlog-core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
index 126d337..df5284d 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
@@ -17,6 +17,9 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.exceptions.LogEmptyException;
import org.apache.distributedlog.exceptions.LogNotFoundException;
@@ -25,13 +28,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-import static org.junit.Assert.*;
+
/**
- * Utils for non blocking reads tests
+ * Utils for non blocking reads tests.
*/
class NonBlockingReadsTestUtil {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
index ae77522..80ef233 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
@@ -17,20 +17,20 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
import java.util.Arrays;
-
import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
-
-import org.apache.distributedlog.exceptions.EndOfStreamException;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
+/**
+ * Test Case for {@link AppendOnlyStreamReader}.
+ */
public class TestAppendOnlyStreamReader extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestAppendOnlyStreamReader.class);
@@ -94,8 +94,8 @@
DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
confLocal.loadConf(conf);
confLocal.setImmediateFlushEnabled(false);
- confLocal.setOutputBufferSize(1024*100);
- confLocal.setPeriodicFlushFrequencyMilliSeconds(1000*60);
+ confLocal.setOutputBufferSize(1024 * 100);
+ confLocal.setPeriodicFlushFrequencyMilliSeconds(1000 * 60);
skipForwardThenSkipBack(name, confLocal);
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
index fc1f241..79f94f0 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
@@ -17,6 +17,8 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
+
import java.io.ByteArrayInputStream;
import java.net.URI;
@@ -24,20 +26,21 @@
import java.util.concurrent.TimeUnit;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.util.FailpointUtils;
import org.apache.distributedlog.util.Utils;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
-
-import org.apache.distributedlog.exceptions.EndOfStreamException;
-import org.apache.distributedlog.exceptions.WriteException;
-import org.apache.distributedlog.util.FailpointUtils;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
+
+/**
+ * Test Case for {@link AppendOnlyStreamWriter}.
+ */
public class TestAppendOnlyStreamWriter extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestAppendOnlyStreamWriter.class);
@@ -113,7 +116,7 @@
public void testPositionUpdatesOnlyAfterWriteCompletion() throws Exception {
String name = testNames.getMethodName();
DistributedLogConfiguration conf = new DistributedLogConfiguration();
- conf.setPeriodicFlushFrequencyMilliSeconds(10*1000);
+ conf.setPeriodicFlushFrequencyMilliSeconds(10 * 1000);
conf.setImmediateFlushEnabled(false);
DistributedLogManager dlmwriter = createNewDLM(conf, name);
@@ -154,9 +157,9 @@
DistributedLogConfiguration conf = new DistributedLogConfiguration();
// Long flush time, but we don't wait for it.
- conf.setPeriodicFlushFrequencyMilliSeconds(100*1000);
+ conf.setPeriodicFlushFrequencyMilliSeconds(100 * 1000);
conf.setImmediateFlushEnabled(false);
- conf.setOutputBufferSize(1024*1024);
+ conf.setOutputBufferSize(1024 * 1024);
DistributedLogManager dlmwriter = createNewDLM(conf, name);
byte[] byteStream = DLMTestUtil.repeatString("abc", 11).getBytes();
@@ -178,9 +181,9 @@
public void testPositionUpdatesOnlyAfterWriteCompletionWithoutFsync() throws Exception {
String name = testNames.getMethodName();
DistributedLogConfiguration conf = new DistributedLogConfiguration();
- conf.setPeriodicFlushFrequencyMilliSeconds(1*1000);
+ conf.setPeriodicFlushFrequencyMilliSeconds(1 * 1000);
conf.setImmediateFlushEnabled(false);
- conf.setOutputBufferSize(1024*1024);
+ conf.setOutputBufferSize(1024 * 1024);
DistributedLogManager dlmwriter = createNewDLM(conf, name);
byte[] byteStream = DLMTestUtil.repeatString("abc", 11).getBytes();
@@ -221,14 +224,14 @@
String name = testNames.getMethodName();
DistributedLogConfiguration conf = new DistributedLogConfiguration();
conf.setImmediateFlushEnabled(false);
- conf.setPeriodicFlushFrequencyMilliSeconds(60*1000);
- conf.setOutputBufferSize(1024*1024);
+ conf.setPeriodicFlushFrequencyMilliSeconds(60 * 1000);
+ conf.setOutputBufferSize(1024 * 1024);
conf.setLogSegmentSequenceNumberValidationEnabled(false);
- final int WRITE_LEN = 5;
- final int SECTION_WRITES = 10;
- long read = writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(conf, name, WRITE_LEN, SECTION_WRITES);
- assertEquals((2*SECTION_WRITES + 1)*WRITE_LEN, read);
+ final int writeLen = 5;
+ final int sectionWrites = 10;
+ long read = writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(conf, name, writeLen, sectionWrites);
+ assertEquals((2 * sectionWrites + 1) * writeLen, read);
}
@Test(timeout = 60000)
@@ -236,18 +239,17 @@
String name = testNames.getMethodName();
DistributedLogConfiguration conf = new DistributedLogConfiguration();
conf.setImmediateFlushEnabled(false);
- conf.setPeriodicFlushFrequencyMilliSeconds(60*1000);
- conf.setOutputBufferSize(1024*1024);
+ conf.setPeriodicFlushFrequencyMilliSeconds(60 * 1000);
+ conf.setOutputBufferSize(1024 * 1024);
conf.setDisableRollingOnLogSegmentError(true);
- final int WRITE_LEN = 5;
- final int SECTION_WRITES = 10;
+ final int writeLen = 5;
+ final int sectionWrites = 10;
try {
- writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(conf, name, WRITE_LEN, SECTION_WRITES);
+ writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(conf, name, writeLen, sectionWrites);
fail("should have thrown");
} catch (BKTransmitException ex) {
- ;
}
BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
@@ -278,8 +280,8 @@
}
writer.force(false);
- long read = read(dlm, 1*sectionWrites*writeLen);
- assertEquals(1*sectionWrites*writeLen, read);
+ long read = read(dlm, 1 * sectionWrites * writeLen);
+ assertEquals(1 * sectionWrites * writeLen, read);
// Now write another 100, but trigger failure during transmit.
for (int i = 0; i < sectionWrites; i++) {
@@ -294,7 +296,6 @@
writer.force(false);
fail("should have thown ⊙﹏⊙");
} catch (WriteException we) {
- ;
} finally {
FailpointUtils.removeFailpoint(
FailpointUtils.FailPointName.FP_TransmitFailGetBuffer);
@@ -313,7 +314,7 @@
writer.close();
long length = dlm.getLastTxId();
- assertEquals(3*sectionWrites*writeLen+5, length);
+ assertEquals(3 * sectionWrites * writeLen + 5, length);
read = read(dlm, length);
dlm.close();
return read;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
index 6efd0c1..0953337 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
@@ -17,6 +17,15 @@
*/
package org.apache.distributedlog;
+
+import static org.apache.distributedlog.DLMTestUtil.validateFutureFailed;
+import static org.apache.distributedlog.DLMTestUtil.validateFutureSucceededAndGetResult;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.junit.Assert.*;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import org.apache.distributedlog.api.DistributedLogManager;
@@ -31,15 +40,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import static org.apache.distributedlog.DLMTestUtil.validateFutureFailed;
-import static org.apache.distributedlog.DLMTestUtil.validateFutureSucceededAndGetResult;
-import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
-import static org.junit.Assert.*;
+
/**
* Test cases for bulk writes.
@@ -61,9 +63,7 @@
/**
* Test Case for partial failure in a bulk write.
- *
* Write a batch: 10 good records + 1 too large record + 10 good records.
- *
* Expected: first 10 good records succeed, the too-large-record will be rejected, while
* the last 10 good records will be cancelled because their previous write is rejected.
*/
@@ -74,7 +74,7 @@
confLocal.loadConf(testConf);
confLocal.setOutputBufferSize(1024);
DistributedLogManager dlm = createNewDLM(confLocal, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
final int goodRecs = 10;
@@ -87,7 +87,7 @@
List<CompletableFuture<DLSN>> results = validateFutureSucceededAndGetResult(futureResults);
// One future returned for each write.
- assertEquals(2*goodRecs + 1, results.size());
+ assertEquals(2 * goodRecs + 1, results.size());
// First goodRecs are good.
for (int i = 0; i < goodRecs; i++) {
@@ -98,7 +98,7 @@
validateFutureFailed(results.get(goodRecs), LogRecordTooLongException.class);
// Rest are WriteCancelledException.
- for (int i = goodRecs+1; i < 2*goodRecs+1; i++) {
+ for (int i = goodRecs + 1; i < 2 * goodRecs + 1; i++) {
validateFutureFailed(results.get(i), WriteCancelledException.class);
}
@@ -108,7 +108,6 @@
/**
* Test Case for a total failure in a bulk write.
- *
* Write 100 records as a batch. Inject failure on transmit and all records should be failed.
*
* @throws Exception
@@ -120,7 +119,7 @@
confLocal.loadConf(testConf);
confLocal.setOutputBufferSize(1024);
DistributedLogManager dlm = createNewDLM(confLocal, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
final int batchSize = 100;
FailpointUtils.setFailpoint(
@@ -155,7 +154,7 @@
confLocal.setMaxLogSegmentBytes(1024);
confLocal.setLogSegmentRollingIntervalMinutes(0);
DistributedLogManager dlm = createNewDLM(confLocal, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
// Write one record larger than max seg size. Ledger doesn't roll until next write.
int txid = 1;
@@ -198,7 +197,7 @@
confLocal.loadConf(testConf);
confLocal.setOutputBufferSize(1024);
DistributedLogManager dlm = createNewDLM(confLocal, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
int batchSize = 100;
int recSize = 1024;
@@ -222,7 +221,7 @@
slotIndex = 0;
txIndex += batchSize;
writer.closeAndComplete();
- writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
writer.closeAndComplete();
@@ -240,12 +239,12 @@
confLocal.loadConf(testConf);
confLocal.setOutputBufferSize(1024);
DistributedLogManager dlm = createNewDLM(confLocal, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
// First entry.
int numTransmissions = 4;
- int recSize = 10*1024;
- int batchSize = (numTransmissions*MAX_LOGRECORDSET_SIZE+1)/recSize;
+ int recSize = 10 * 1024;
+ int batchSize = (numTransmissions * MAX_LOGRECORDSET_SIZE + 1) / recSize;
long ledgerIndex = 1;
long entryIndex = 0;
long slotIndex = 0;
@@ -269,12 +268,12 @@
confLocal.loadConf(testConf);
confLocal.setOutputBufferSize(1024);
DistributedLogManager dlm = createNewDLM(confLocal, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
// First entry.
int numTransmissions = 4;
- int recSize = 10*1024;
- int batchSize = (numTransmissions*MAX_LOGRECORDSET_SIZE+1)/recSize;
+ int recSize = 10 * 1024;
+ int batchSize = (numTransmissions * MAX_LOGRECORDSET_SIZE + 1) / recSize;
long ledgerIndex = 1;
long entryIndex = 0;
long slotIndex = 0;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
index 62ac5ef..aec1b1f 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
@@ -17,6 +17,10 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.net.URI;
import java.util.ArrayList;
import java.util.concurrent.CancellationException;
@@ -30,15 +34,15 @@
import org.apache.distributedlog.api.AsyncLogReader;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
import org.apache.distributedlog.exceptions.LockCancelledException;
import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
import org.apache.distributedlog.impl.BKNamespaceDriver;
import org.apache.distributedlog.lock.LockClosedException;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.namespace.NamespaceDriver;
-import org.apache.distributedlog.api.subscription.SubscriptionsStore;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
import org.apache.distributedlog.util.Utils;
import org.junit.Rule;
import org.junit.Test;
@@ -46,11 +50,12 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
+
+
+/**
+ * TestAsyncReaderLock.
+ */
public class TestAsyncReaderLock extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestAsyncReaderLock.class);
@@ -70,7 +75,7 @@
public void testReaderLockIfLockPathDoesntExist() throws Exception {
final String name = runtime.getMethodName();
DistributedLogManager dlm = createNewDLM(conf, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.closeAndComplete();
@@ -99,7 +104,7 @@
public void testReaderLockCloseInAcquireCallback() throws Exception {
final String name = runtime.getMethodName();
DistributedLogManager dlm = createNewDLM(conf, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.closeAndComplete();
@@ -122,7 +127,7 @@
public void testReaderLockBackgroundReaderLockAcquire() throws Exception {
final String name = runtime.getMethodName();
DistributedLogManager dlm = createNewDLM(conf, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.closeAndComplete();
@@ -179,14 +184,15 @@
public void testReaderLockManyLocks() throws Exception {
String name = runtime.getMethodName();
DistributedLogManager dlm = createNewDLM(conf, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.write(DLMTestUtil.getLogRecordInstance(2L));
writer.closeAndComplete();
int count = 5;
final CountDownLatch acquiredLatch = new CountDownLatch(count);
- final ArrayList<CompletableFuture<AsyncLogReader>> readers = new ArrayList<CompletableFuture<AsyncLogReader>>(count);
+ final ArrayList<CompletableFuture<AsyncLogReader>> readers =
+ new ArrayList<CompletableFuture<AsyncLogReader>>(count);
for (int i = 0; i < count; i++) {
readers.add(null);
}
@@ -219,7 +225,7 @@
public void testReaderLockDlmClosed() throws Exception {
String name = runtime.getMethodName();
DistributedLogManager dlm0 = createNewDLM(conf, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm0.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.write(DLMTestUtil.getLogRecordInstance(2L));
writer.closeAndComplete();
@@ -255,7 +261,7 @@
.uri(uri)
.build();
DistributedLogManager dlm0 = ns0.openLog(name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm0.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.write(DLMTestUtil.getLogRecordInstance(2L));
writer.closeAndComplete();
@@ -267,7 +273,8 @@
DistributedLogManager dlm1 = ns1.openLog(name);
CompletableFuture<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
AsyncLogReader reader1 = Utils.ioResult(futureReader1);
- ZooKeeperClientUtils.expireSession(((BKNamespaceDriver) ns1.getNamespaceDriver()).getWriterZKC(), zkServers, 1000);
+ ZooKeeperClientUtils.expireSession(((BKNamespaceDriver)
+ ns1.getNamespaceDriver()).getWriterZKC(), zkServers, 1000);
// The result of expireSession is somewhat non-deterministic with this lock.
// It may fail with LockingException or it may succesfully reacquire, so for
@@ -293,7 +300,7 @@
public void testReaderLockFutureCancelledWhileWaiting() throws Exception {
String name = runtime.getMethodName();
DistributedLogManager dlm0 = createNewDLM(conf, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm0.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.write(DLMTestUtil.getLogRecordInstance(2L));
writer.closeAndComplete();
@@ -328,7 +335,7 @@
public void testReaderLockFutureCancelledWhileLocked() throws Exception {
String name = runtime.getMethodName();
DistributedLogManager dlm0 = createNewDLM(conf, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm0.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.write(DLMTestUtil.getLogRecordInstance(2L));
writer.closeAndComplete();
@@ -350,7 +357,7 @@
public void testReaderLockSharedDlmDoesNotConflict() throws Exception {
String name = runtime.getMethodName();
DistributedLogManager dlm0 = createNewDLM(conf, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm0.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.write(DLMTestUtil.getLogRecordInstance(2L));
writer.closeAndComplete();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
index c1181e8..2efdca0 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
@@ -17,6 +17,12 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Charsets.UTF_8;
+import static org.apache.distributedlog.DLMTestUtil.validateFutureFailed;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.junit.Assert.*;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Lists;
import io.netty.buffer.ByteBuf;
import java.io.IOException;
import java.net.URI;
@@ -29,32 +35,47 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
+import junit.framework.Assert;
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import org.apache.distributedlog.api.AsyncLogReader;
-import org.apache.distributedlog.api.AsyncLogWriter;
-import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.api.LogReader;
-import org.apache.distributedlog.api.LogWriter;
-import org.apache.distributedlog.api.namespace.Namespace;
-import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
-import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
-import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
-import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.distributedlog.exceptions.BKTransmitException;
-import org.apache.distributedlog.exceptions.LockingException;
-import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.io.CompressionCodec;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.util.Utils;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.BookKeeperAccessor;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.LedgerMetadata;
import org.apache.bookkeeper.feature.FixedValueFeature;
import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
+
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.api.LogWriter;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.IdleReaderException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.exceptions.ReadCancelledException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.util.SimplePermitLimiter;
+
+
+import org.apache.distributedlog.util.Utils;
+
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
@@ -62,26 +83,12 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.base.Stopwatch;
-import org.apache.distributedlog.exceptions.DLIllegalStateException;
-import org.apache.distributedlog.exceptions.EndOfStreamException;
-import org.apache.distributedlog.exceptions.IdleReaderException;
-import org.apache.distributedlog.exceptions.LogRecordTooLongException;
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.exceptions.ReadCancelledException;
-import org.apache.distributedlog.exceptions.WriteException;
-import org.apache.distributedlog.lock.DistributedLock;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
-import org.apache.distributedlog.util.SimplePermitLimiter;
-import junit.framework.Assert;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.apache.distributedlog.DLMTestUtil.validateFutureFailed;
-import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static org.junit.Assert.*;
+
+/**
+ * Test Cases for AsyncReaderWriter.
+ */
public class TestAsyncReaderWriter extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestAsyncReaderWriter.class);
@@ -113,7 +120,7 @@
int txid = 1;
for (long i = 0; i < 3; i++) {
final long currentLogSegmentSeqNo = i + 1;
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
DLSN dlsn = Utils.ioResult(writer.writeControlRecord(new LogRecord(txid++, "control".getBytes(UTF_8))));
assertEquals(currentLogSegmentSeqNo, dlsn.getLogSegmentSequenceNo());
assertEquals(0, dlsn.getEntryId());
@@ -159,7 +166,7 @@
confLocal.setMaxLogSegmentBytes(1024);
confLocal.setLogSegmentRollingIntervalMinutes(0);
DistributedLogManager dlm = createNewDLM(confLocal, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
// Write one record larger than max seg size. Ledger doesn't roll until next write.
int txid = 1;
@@ -206,7 +213,7 @@
int txid = 1;
for (long i = 0; i < numLogSegments; i++) {
final long currentLogSegmentSeqNo = i + 1;
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
for (long j = 0; j < numRecordsPerLogSegment; j++) {
final long currentEntryId = j;
final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
@@ -214,12 +221,13 @@
dlsnFuture.whenComplete(new FutureEventListener<DLSN>() {
@Override
public void onSuccess(DLSN value) {
- if(value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) {
- LOG.debug("LogSegmentSequenceNumber: {}, Expected {}", value.getLogSegmentSequenceNo(), currentLogSegmentSeqNo);
+ if (value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) {
+ LOG.debug("LogSegmentSequenceNumber: {}, Expected {}",
+ value.getLogSegmentSequenceNo(), currentLogSegmentSeqNo);
errorsFound.set(true);
}
- if(value.getEntryId() != currentEntryId) {
+ if (value.getEntryId() != currentEntryId) {
LOG.debug("EntryId: {}, Expected {}", value.getEntryId(), currentEntryId);
errorsFound.set(true);
}
@@ -233,7 +241,8 @@
}
@Override
public void onFailure(Throwable cause) {
- LOG.error("Encountered exception on writing record {} in log segment {}", currentEntryId, currentLogSegmentSeqNo);
+ LOG.error("Encountered exception on writing record {} in log segment {}",
+ currentEntryId, currentLogSegmentSeqNo);
errorsFound.set(true);
}
});
@@ -255,7 +264,8 @@
}
/**
- * Write records into <i>numLogSegments</i> log segments. Each log segment has <i>numRecordsPerLogSegment</i> records.
+ * Write records into <i>numLogSegments</i> log segments.
+ * Each log segment has <i>numRecordsPerLogSegment</i> records.
*
* @param dlm
* distributedlog manager
@@ -274,7 +284,7 @@
boolean emptyRecord) throws IOException {
long txid = startTxId;
for (long i = 0; i < numLogSegments; i++) {
- BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= numRecordsPerLogSegment; j++) {
if (emptyRecord) {
writer.write(DLMTestUtil.getEmptyLogRecordInstance(txid++));
@@ -315,7 +325,7 @@
} else {
writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
}
- if (j % flushPerNumRecords == 0 ) {
+ if (j % flushPerNumRecords == 0) {
writer.flush();
writer.commit();
}
@@ -552,7 +562,7 @@
}
/**
- * Test Async Read by positioning to a given position in the log
+ * Test Async Read by positioning to a given position in the log.
* @throws Exception
*/
@Test(timeout = 60000)
@@ -574,7 +584,7 @@
// write another log segment with 5 records
txid = writeLogSegment(dlm, 5, txid, Integer.MAX_VALUE, false);
- final CountDownLatch syncLatch = new CountDownLatch((int)(txid - 14));
+ final CountDownLatch syncLatch = new CountDownLatch((int) (txid - 14));
final CountDownLatch doneLatch = new CountDownLatch(1);
final AtomicBoolean errorsFound = new AtomicBoolean(false);
final AsyncLogReader reader = dlm.getAsyncLogReader(new DLSN(2, 2, 4));
@@ -620,8 +630,7 @@
/**
* Test if entries written using log segment metadata that doesn't support enveloping
* can be read correctly by a reader supporting both.
- *
- * NOTE: An older reader cannot read enveloped entry, so we don't have a test case covering
+ *NOTE: An older reader cannot read enveloped entry, so we don't have a test case covering
* the other scenario.
*
* @throws Exception
@@ -655,16 +664,16 @@
}
/**
- * Invoked if the computation completes successfully
+ * Invoked if the computation completes successfully.
*/
@Override
public void onSuccess(DLSN value) {
- if(value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) {
+ if (value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) {
LOG.error("Ledger Seq No: {}, Expected: {}", value.getLogSegmentSequenceNo(), currentLogSegmentSeqNo);
errorsFound.set(true);
}
- if(verifyEntryId && value.getEntryId() != currentEntryId) {
+ if (verifyEntryId && value.getEntryId() != currentEntryId) {
LOG.error("EntryId: {}, Expected: {}", value.getEntryId(), currentEntryId);
errorsFound.set(true);
}
@@ -672,7 +681,7 @@
}
/**
- * Invoked if the computation completes unsuccessfully
+ * Invoked if the computation completes unsuccessfully.
*/
@Override
public void onFailure(Throwable cause) {
@@ -714,7 +723,7 @@
int txid = 1;
for (long i = 0; i < 3; i++) {
final long currentLogSegmentSeqNo = i + 1;
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
for (long j = 0; j < 10; j++) {
final long currentEntryId = j;
final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
@@ -792,7 +801,7 @@
int txid = 1;
for (long i = 0; i < 3; i++) {
final long currentLogSegmentSeqNo = i + 1;
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
for (long j = 0; j < 10; j++) {
final long currentEntryId = j;
final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
@@ -865,7 +874,7 @@
final long currentLogSegmentSeqNo = i + 1;
BKAsyncLogWriter[] writers = new BKAsyncLogWriter[count];
for (int s = 0; s < count; s++) {
- writers[s] = (BKAsyncLogWriter)(dlms[s].startAsyncLogSegmentNonPartitioned());
+ writers[s] = (BKAsyncLogWriter) (dlms[s].startAsyncLogSegmentNonPartitioned());
}
for (long j = 0; j < 1; j++) {
final long currentEntryId = j;
@@ -899,7 +908,7 @@
}
/**
- * Flaky test fixed: readers need to be added to the pendingReaders
+ * Flaky test fixed: readers need to be added to the pendingReaders.
* @throws Exception
*/
@Test(timeout = 300000)
@@ -936,7 +945,7 @@
int txid = 1;
for (long i = 0; i < numLogSegments; i++) {
final long currentLogSegmentSeqNo = i + 1;
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
for (long j = 0; j < numRecordsPerLogSegment; j++) {
final long currentEntryId = j;
final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
@@ -987,7 +996,7 @@
int txid = 1;
for (long i = 0; i < numLogSegments; i++) {
final long currentLogSegmentSeqNo = i + 1;
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
for (long j = 0; j < numRecordsPerLogSegment; j++) {
Thread.sleep(50);
final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
@@ -995,7 +1004,8 @@
dlsnFuture.whenComplete(new WriteFutureEventListener(
record, currentLogSegmentSeqNo, j, writeLatch, writeErrors, false));
if (i == 0 && j == 0) {
- boolean monotonic = LogSegmentMetadata.supportsSequenceId(confLocal.getDLLedgerMetadataLayoutVersion());
+ boolean monotonic =
+ LogSegmentMetadata.supportsSequenceId(confLocal.getDLLedgerMetadataLayoutVersion());
TestAsyncReaderWriter.readNext(
reader,
DLSN.InvalidDLSN,
@@ -1025,7 +1035,7 @@
final String name = "distrlog-cancel-read-requests-on-reader-closed";
DistributedLogManager dlm = createNewDLM(testConf, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.closeAndComplete();
@@ -1082,12 +1092,12 @@
confLocal.setMinDelayBetweenImmediateFlushMs(100);
DistributedLogManager dlm = createNewDLM(confLocal, name);
final Thread currentThread = Thread.currentThread();
- final int COUNT = 5000;
- final CountDownLatch syncLatch = new CountDownLatch(COUNT);
+ final int count = 5000;
+ final CountDownLatch syncLatch = new CountDownLatch(count);
int txid = 1;
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
Stopwatch executionTime = Stopwatch.createStarted();
- for (long i = 0; i < COUNT; i++) {
+ for (long i = 0; i < count; i++) {
Thread.sleep(1);
final LogRecord record = DLMTestUtil.getLogRecordInstance(txid++);
CompletableFuture<DLSN> dlsnFuture = writer.write(record);
@@ -1122,12 +1132,14 @@
assertTrue(success);
LogRecordWithDLSN last = dlm.getLastLogRecord();
- LOG.info("Last Entry {}; elapsed time {}", last.getDlsn().getEntryId(), executionTime.elapsed(TimeUnit.MILLISECONDS));
+ LOG.info("Last Entry {}; elapsed time {}",
+ last.getDlsn().getEntryId(), executionTime.elapsed(TimeUnit.MILLISECONDS));
// Regardless of how many records we wrote; the number of BK entries should always be bounded by the min delay.
- // Since there are two flush processes--data flush and control flush, and since control flush may also end up flushing
- // data if data is available, the upper bound is 2*(time/min_delay + 1)
- assertTrue(last.getDlsn().getEntryId() <= ((executionTime.elapsed(TimeUnit.MILLISECONDS) / confLocal.getMinDelayBetweenImmediateFlushMs() + 1))*2);
+ // Since there are two flush processes--data flush and control flush, and since control flush may also end up
+ // flushing data if data is available, the upper bound is 2*(time/min_delay + 1)
+ assertTrue(last.getDlsn().getEntryId() <= ((executionTime.elapsed(TimeUnit.MILLISECONDS)
+ / confLocal.getMinDelayBetweenImmediateFlushMs() + 1)) * 2);
DLMTestUtil.verifyLogRecord(last);
dlm.close();
@@ -1153,7 +1165,7 @@
DistributedLogManager dlm1 = namespace1.openLog(name);
int txid = 1;
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
// First write succeeds since lock isnt checked until transmit, which is scheduled
Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
@@ -1164,7 +1176,7 @@
Utils.ioResult(lock.asyncClose());
// Get second writer, steal lock
- BKAsyncLogWriter writer2 = (BKAsyncLogWriter)(dlm1.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer2 = (BKAsyncLogWriter) (dlm1.startAsyncLogSegmentNonPartitioned());
try {
// Succeeds, kicks off scheduked flush
@@ -1196,7 +1208,7 @@
} else {
dlm = createNewDLM(confLocal, runtime.getMethodName());
}
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
ArrayList<CompletableFuture<DLSN>> results = new ArrayList<CompletableFuture<DLSN>>(1000);
for (int i = 0; i < 1000; i++) {
results.add(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
@@ -1244,7 +1256,7 @@
confLocal.setPerWriterOutstandingWriteLimit(0);
confLocal.setOutstandingWriteLimitDarkmode(true);
DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
ArrayList<CompletableFuture<DLSN>> results = new ArrayList<CompletableFuture<DLSN>>(1000);
for (int i = 0; i < 1000; i++) {
results.add(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
@@ -1265,7 +1277,7 @@
confLocal.setImmediateFlushEnabled(true);
BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
long txId = 1L;
for (int i = 0; i < 5; i++) {
@@ -1309,7 +1321,7 @@
confLocal.setImmediateFlushEnabled(true);
BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
- BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
long txId = 1L;
for (int i = 0; i < 5; i++) {
@@ -1366,7 +1378,7 @@
int txid = 1;
for (long i = 0; i < numSegments; i++) {
long start = txid;
- BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= segmentSize; j++) {
writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
if ((i == 0) && (j == 1)) {
@@ -1381,7 +1393,7 @@
- 200 // BK commitTime
- 100; //safety margin
- for (int iter = 1; iter <= (2 * idleReaderErrorThreshold / threadSleepTime) ; iter++) {
+ for (int iter = 1; iter <= (2 * idleReaderErrorThreshold / threadSleepTime); iter++) {
Thread.sleep(threadSleepTime);
writer.write(DLMTestUtil.getLargeLogRecordInstance(txid, true));
writer.flush();
@@ -1466,7 +1478,7 @@
DistributedLogManager dlm = createNewDLM(confLocal, name);
BKAsyncLogWriter writer =
- (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+ (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
writer.abort();
@@ -1545,7 +1557,7 @@
}, 0, TimeUnit.MILLISECONDS);
latch.await();
- BKAsyncLogReader reader = (BKAsyncLogReader)dlm.getAsyncLogReader(DLSN.InitialDLSN);
+ BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
reader.disableReadAheadLogSegmentsNotification();
boolean exceptionEncountered = false;
int recordCount = 0;
@@ -1656,9 +1668,9 @@
DistributedLogManager dlm = createNewDLM(confLocal, name);
BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
- final int NUM_RECORDS = 10;
+ final int numRecords = 10;
int i = 1;
- for (; i <= NUM_RECORDS; i++) {
+ for (; i <= numRecords; i++) {
Utils.ioResult(writer.write(DLMTestUtil.getLogRecordInstance(i)));
assertEquals("last tx id should become " + i,
i, writer.getLastTxId());
@@ -1677,9 +1689,9 @@
BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
LogRecord record = null;
- for (int j = 0; j < NUM_RECORDS; j++) {
+ for (int j = 0; j < numRecords; j++) {
record = Utils.ioResult(reader.readNext());
- assertEquals(j+1, record.getTransactionId());
+ assertEquals(j + 1, record.getTransactionId());
}
try {
@@ -1735,7 +1747,8 @@
Utils.ioResult(writer.write(controlRecord));
BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
- CompletableFuture<List<LogRecordWithDLSN>> bulkReadFuture = reader.readBulk(2, Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+ CompletableFuture<List<LogRecordWithDLSN>> bulkReadFuture =
+ reader.readBulk(2, Long.MAX_VALUE, TimeUnit.MILLISECONDS);
CompletableFuture<LogRecordWithDLSN> readFuture = reader.readNext();
// write another records
@@ -1983,8 +1996,8 @@
List<LogSegmentMetadata> segments = dlm.getLogSegments();
assertEquals(1, segments.size());
long ledgerId = segments.get(0).getLogSegmentId();
- LedgerHandle lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC()
- .get().openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
+ LedgerHandle lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get()
+ .openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
LedgerMetadata metadata = BookKeeperAccessor.getLedgerMetadata(lh);
assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT, metadata.getEnsembleSize());
lh.close();
@@ -2002,8 +2015,8 @@
segments = dlm.getLogSegments();
assertEquals(1, segments.size());
ledgerId = segments.get(0).getLogSegmentId();
- lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC()
- .get().openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
+ lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get()
+ .openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
metadata = BookKeeperAccessor.getLedgerMetadata(lh);
assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT - 1, metadata.getEnsembleSize());
lh.close();
@@ -2106,8 +2119,8 @@
assertEquals(new DLSN(1L, 5L, i - 5), record.getDlsn());
assertEquals(6L, record.getTransactionId());
}
- assertEquals(i+1, record.getPositionWithinLogSegment());
- assertArrayEquals(DLMTestUtil.generatePayload(i+1), record.getPayload());
+ assertEquals(i + 1, record.getPositionWithinLogSegment());
+ assertArrayEquals(DLMTestUtil.generatePayload(i + 1), record.getPayload());
}
Utils.close(reader1);
readDLM1.close();
@@ -2125,7 +2138,7 @@
assertEquals(new DLSN(1L, i, 0L), record.getDlsn());
assertEquals(1L + i, record.getTransactionId());
assertEquals(i + 1, record.getPositionWithinLogSegment());
- assertArrayEquals(DLMTestUtil.generatePayload(i+1), record.getPayload());
+ assertArrayEquals(DLMTestUtil.generatePayload(i + 1), record.getPayload());
} else if (i >= 6L) {
assertEquals(new DLSN(1L, 6L + i - 6, 0L), record.getDlsn());
assertEquals(11L + i - 6, record.getTransactionId());
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
index 5fd8fe3..c133bb8 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
@@ -17,6 +17,10 @@
*/
package org.apache.distributedlog;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.net.URI;
import java.util.Collection;
@@ -29,7 +33,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
-
+import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.distributedlog.api.AsyncLogReader;
import org.apache.distributedlog.api.AsyncLogWriter;
@@ -38,18 +42,27 @@
import org.apache.distributedlog.api.LogWriter;
import org.apache.distributedlog.api.MetadataAccessor;
import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.api.subscription.SubscriptionsStore;
+import org.apache.distributedlog.callback.LogSegmentListener;
import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.InvalidStreamNameException;
import org.apache.distributedlog.exceptions.LogEmptyException;
import org.apache.distributedlog.exceptions.LogNotFoundException;
import org.apache.distributedlog.exceptions.LogReadException;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
import org.apache.distributedlog.impl.BKNamespaceDriver;
import org.apache.distributedlog.impl.ZKLogSegmentMetadataStore;
import org.apache.distributedlog.io.Abortables;
import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
+import org.apache.distributedlog.metadata.LogMetadata;
+import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
+import org.apache.distributedlog.metadata.MetadataUpdater;
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Utils;
-import org.apache.bookkeeper.client.BKException;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@@ -57,21 +70,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.distributedlog.callback.LogSegmentListener;
-import org.apache.distributedlog.exceptions.EndOfStreamException;
-import org.apache.distributedlog.exceptions.InvalidStreamNameException;
-import org.apache.distributedlog.exceptions.LogRecordTooLongException;
-import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
-import org.apache.distributedlog.metadata.LogMetadata;
-import org.apache.distributedlog.metadata.MetadataUpdater;
-import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.api.subscription.SubscriptionsStore;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
-import static org.junit.Assert.assertEquals;
+/**
+ * Test Cases for {@link DistributedLogManager}.
+ */
public class TestBKDistributedLogManager extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestBKDistributedLogManager.class);
@@ -96,7 +99,7 @@
writer.closeAndComplete();
BKLogWriteHandler blplm = dlm.createWriteHandler(true);
assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
- perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+ perStreamLogWriter.getLogSegmentSequenceNumber()), false));
Utils.ioResult(blplm.asyncClose());
}
@@ -144,7 +147,7 @@
public void testNumberOfTransactions() throws Exception {
String name = "distrlog-txncount";
DistributedLogManager dlm = createNewDLM(conf, name);
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long i = 1; i <= 100; i++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(i);
out.write(op);
@@ -212,7 +215,7 @@
public void testWriteRestartFrom1() throws Exception {
DistributedLogManager dlm = createNewDLM(conf, "distrlog-restartFrom1");
long txid = 1;
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
out.write(op);
@@ -221,7 +224,7 @@
txid = 1;
try {
- out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
out.write(DLMTestUtil.getLogRecordInstance(txid));
fail("Shouldn't be able to start another journal from " + txid
+ " when one already exists");
@@ -234,7 +237,7 @@
// test border case
txid = DEFAULT_SEGMENT_SIZE - 1;
try {
- out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
out.write(DLMTestUtil.getLogRecordInstance(txid));
fail("Shouldn't be able to start another journal from " + txid
+ " when one already exists");
@@ -246,7 +249,7 @@
// open journal continuing from before
txid = DEFAULT_SEGMENT_SIZE + 1;
- out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
assertNotNull(out);
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
@@ -257,7 +260,7 @@
// open journal arbitarily far in the future
txid = DEFAULT_SEGMENT_SIZE * 4;
- out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
out.write(DLMTestUtil.getLogRecordInstance(txid));
out.close();
dlm.close();
@@ -290,7 +293,7 @@
String name = "distrlog-simpleread";
DistributedLogManager dlm = createNewDLM(conf, name);
final long numTransactions = 10000;
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long i = 1; i <= numTransactions; i++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(i);
out.write(op);
@@ -308,7 +311,7 @@
long txid = 1;
for (long i = 0; i < 3; i++) {
long start = txid;
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
out.write(op);
@@ -321,7 +324,7 @@
perStreamLogWriter.getLogSegmentSequenceNumber()), false));
Utils.ioResult(blplm.asyncClose());
}
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
out.write(op);
@@ -340,7 +343,7 @@
DistributedLogManager dlm = createNewDLM(conf, name);
long txid = 1;
for (long i = 0; i < 3; i++) {
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
out.write(op);
@@ -348,7 +351,7 @@
out.closeAndComplete();
}
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
out.write(op);
@@ -384,7 +387,7 @@
long txid = 1;
for (long i = 0; i < 3; i++) {
long start = txid;
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
out.write(op);
@@ -405,7 +408,7 @@
Utils.ioResult(blplm.asyncClose());
}
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
out.write(op);
@@ -478,7 +481,7 @@
int logCount = 0;
Iterator<String> logIter = namespace.getLogs();
- while(logIter.hasNext()) {
+ while (logIter.hasNext()) {
String log = logIter.next();
logCount++;
assertEquals(name, log);
@@ -566,7 +569,7 @@
private long writeAndMarkEndOfStream(DistributedLogManager dlm, long txid) throws Exception {
for (long i = 0; i < 3; i++) {
long start = txid;
- BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
writer.write(DLMTestUtil.getLogRecordInstance(txid++));
}
@@ -577,13 +580,13 @@
writer.closeAndComplete();
BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
- perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+ perStreamLogWriter.getLogSegmentSequenceNumber()), false));
Utils.ioResult(blplm.asyncClose());
} else {
writer.markEndOfStream();
BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, DistributedLogConstants.MAX_TXID,
- perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+ perStreamLogWriter.getLogSegmentSequenceNumber()), false));
Utils.ioResult(blplm.asyncClose());
}
}
@@ -672,7 +675,7 @@
DistributedLogManager dlm = createNewDLM(conf, name);
DLMTestUtil.generateCompletedLogSegments(dlm, conf, numCompletedSegments, DEFAULT_SEGMENT_SIZE);
- BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
writer.markEndOfStream();
LogReader reader = dlm.getInputStream(1);
@@ -747,7 +750,7 @@
long txid = 1;
for (long i = 0; i < 3; i++) {
long start = txid;
- BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
writer.write(DLMTestUtil.getLogRecordInstance(txid++));
}
@@ -757,7 +760,7 @@
writer.closeAndComplete();
BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
- perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+ perStreamLogWriter.getLogSegmentSequenceNumber()), false));
Utils.ioResult(blplm.asyncClose());
}
@@ -809,7 +812,7 @@
long txid = 1;
for (long i = 0; i < 3; i++) {
long start = txid;
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
out.write(op);
@@ -830,7 +833,7 @@
Utils.ioResult(blplm.asyncClose());
}
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
LogRecord op = DLMTestUtil.getLogRecordInstance(txid);
op.setControl();
out.write(op);
@@ -899,7 +902,7 @@
LOG.info("Waiting for creating log segment {}.", i);
latches[i].await();
LOG.info("Creating log segment {}.", i);
- BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
LOG.info("Created log segment {}.", i);
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
@@ -1152,7 +1155,8 @@
segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
LogMetadata.getLogSegmentsPath(uri, name, conf.getUnpartitionedStreamName()));
- Assert.assertTrue(segmentList.get(truncDLSN.getLogSegmentSequenceNo()).getMinActiveDLSN().compareTo(truncDLSN) == 0);
+ Assert.assertTrue(segmentList.get(truncDLSN.getLogSegmentSequenceNo())
+ .getMinActiveDLSN().compareTo(truncDLSN) == 0);
{
LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
@@ -1213,7 +1217,7 @@
DistributedLogManager dlm = createNewDLM(conf, name);
long txid = 1;
// Create the log and write some records
- BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+ BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
writer.write(DLMTestUtil.getLogRecordInstance(txid++));
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
index ca3e332..cf6413c 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
@@ -17,6 +17,8 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
+import com.google.common.collect.Sets;
import java.io.IOException;
import java.net.URI;
import java.util.Collection;
@@ -26,19 +28,17 @@
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
-
-import com.google.common.collect.Sets;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogReader;
import org.apache.distributedlog.api.LogWriter;
import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.callback.NamespaceListener;
import org.apache.distributedlog.exceptions.AlreadyClosedException;
import org.apache.distributedlog.exceptions.InvalidStreamNameException;
import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.exceptions.ZKException;
import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.util.DLUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -54,8 +54,10 @@
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
+/**
+ * Test Cases for {@link Namespace}.
+ */
public class TestBKDistributedLogNamespace extends TestDistributedLogBase {
@Rule
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
index 4915137..15711e7 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
@@ -17,21 +17,22 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
import com.google.common.base.Optional;
+import java.util.ArrayList;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.distributedlog.api.AsyncLogWriter;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogWriter;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.LogNotFoundException;
import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
import org.apache.distributedlog.logsegment.LogSegmentFilter;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.Utils;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.concurrent.TimeUnit;
import org.junit.Rule;
import org.junit.Test;
@@ -39,10 +40,9 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
/**
- * Test {@link BKLogReadHandler}
+ * Test {@link BKLogReadHandler}.
*/
public class TestBKLogReadHandler extends TestDistributedLogBase {
@@ -51,7 +51,8 @@
@Rule
public TestName runtime = new TestName();
- private void prepareLogSegmentsNonPartitioned(String name, int numSegments, int numEntriesPerSegment) throws Exception {
+ private void prepareLogSegmentsNonPartitioned(String name,
+ int numSegments, int numEntriesPerSegment) throws Exception {
DistributedLogManager dlm = createNewDLM(conf, name);
long txid = 1;
for (int sid = 0; sid < numSegments; ++sid) {
@@ -92,9 +93,9 @@
Utils.ioResult(out.write(controlRecord));
DLSN last = dlm1.getLastDLSN();
- assertEquals(new DLSN(1,9,0), last);
+ assertEquals(new DLSN(1, 9, 0), last);
DLSN first = Utils.ioResult(dlm1.getFirstDLSNAsync());
- assertEquals(new DLSN(1,0,0), first);
+ assertEquals(new DLSN(1, 0, 0), first);
Utils.close(out);
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
index 4ad0bc0..eca283d 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
@@ -17,7 +17,22 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.feature.SettableFeatureProvider;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.PermitLimiter;
import org.apache.distributedlog.exceptions.BKTransmitException;
import org.apache.distributedlog.exceptions.EndOfStreamException;
import org.apache.distributedlog.exceptions.WriteCancelledException;
@@ -25,21 +40,13 @@
import org.apache.distributedlog.exceptions.ZKException;
import org.apache.distributedlog.impl.BKNamespaceDriver;
import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
import org.apache.distributedlog.lock.SessionLockFactory;
import org.apache.distributedlog.lock.ZKDistributedLock;
import org.apache.distributedlog.lock.ZKSessionLockFactory;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.common.util.PermitLimiter;
import org.apache.distributedlog.util.Utils;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs;
import org.junit.After;
@@ -48,17 +55,11 @@
import org.junit.Test;
import org.junit.rules.TestName;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
+
/**
- * Test Case for BookKeeper Based Log Segment Writer
+ * Test Case for BookKeeper Based Log Segment Writer.
*/
public class TestBKLogSegmentWriter extends TestDistributedLogBase {
@@ -423,7 +424,7 @@
}
/**
- * Close the writer when ledger is fenced: it should release the lock, fail on flushing data and throw exception
+ * Close the writer when ledger is fenced: it should release the lock, fail on flushing data and throw exception.
*
* @throws Exception
*/
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
index c0f208f..7b11ca5 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
@@ -17,25 +17,25 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
+import java.io.IOException;
+import java.net.URI;
import org.apache.distributedlog.api.AsyncLogWriter;
import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.bk.LedgerAllocator;
import org.apache.distributedlog.bk.LedgerAllocatorPool;
import org.apache.distributedlog.impl.BKNamespaceDriver;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.util.FailpointUtils;
import org.apache.distributedlog.util.Utils;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
-import java.io.IOException;
-import java.net.URI;
-import static org.junit.Assert.*;
/**
- * Test {@link BKLogWriteHandler}
+ * Test {@link BKLogWriteHandler}.
*/
public class TestBKLogWriteHandler extends TestDistributedLogBase {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
index 6b2bfad..cb98741 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
@@ -17,6 +17,12 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
+import java.io.IOException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogReader;
import org.apache.distributedlog.exceptions.LogNotFoundException;
@@ -26,20 +32,14 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import static org.junit.Assert.*;
/**
- * Test Sync Log Reader
+ * Test Sync Log Reader.
*/
public class TestBKSyncLogReader extends TestDistributedLogBase {
- static final Logger logger = LoggerFactory.getLogger(TestBKSyncLogReader.class);
+ private static final Logger logger = LoggerFactory.getLogger(TestBKSyncLogReader.class);
@Rule
public TestName testName = new TestName();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java
index e47d020..f92bce9 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java
@@ -29,6 +29,9 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+/**
+ * Test Case for {@link LocalDLMEmulator}.
+ */
public class TestDLMTestUtil {
static final Logger LOG = LoggerFactory.getLogger(TestDLMTestUtil.class);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
index 5b55cd0..f77ff58 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
@@ -21,9 +21,26 @@
import com.google.common.base.Optional;
import com.google.common.base.Ticker;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.feature.SettableFeatureProvider;
+import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.distributedlog.api.LogReader;
import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.common.util.PermitLimiter;
+import org.apache.distributedlog.common.util.SchedulerUtils;
import org.apache.distributedlog.impl.BKNamespaceDriver;
import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
import org.apache.distributedlog.injector.AsyncFailureInjector;
@@ -32,20 +49,9 @@
import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.namespace.NamespaceDriver;
import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.common.util.PermitLimiter;
-import org.apache.distributedlog.common.util.SchedulerUtils;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.tuple.Pair;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
@@ -57,13 +63,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
+/**
+ * DistributedLogBase providing test environment setup for other Test Cases.
+ */
public class TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogBase.class);
@@ -92,7 +95,7 @@
protected static String zkServers;
protected static int zkPort;
protected static int numBookies = 3;
- protected static final List<File> tmpDirs = new ArrayList<File>();
+ private static final List<File> tmpDirs = new ArrayList<File>();
@BeforeClass
public static void setupCluster() throws Exception {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
index e849465..4f52cb2 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
@@ -17,18 +17,20 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
import com.google.common.base.Optional;
-
-import org.apache.distributedlog.net.DNSResolverForRacks;
-import org.apache.distributedlog.net.DNSResolverForRows;
+import java.util.List;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
import org.apache.commons.configuration.StrictConfigurationComparator;
+import org.apache.distributedlog.net.DNSResolverForRacks;
+import org.apache.distributedlog.net.DNSResolverForRows;
import org.junit.Test;
-import java.util.List;
-import static org.junit.Assert.*;
+/**
+ * Test Cases for truncation.
+ */
public class TestDistributedLogConfiguration {
static final class TestDNSResolver implements DNSToSwitchMapping {
@@ -55,12 +57,12 @@
DistributedLogConfiguration.BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT);
DistributedLogConfiguration override = new DistributedLogConfiguration();
override.setPeriodicFlushFrequencyMilliSeconds(
- DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT+1);
+ DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT + 1);
override.setReaderIdleErrorThresholdMillis(
DistributedLogConfiguration.BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT - 1);
conf.loadStreamConf(Optional.of(override));
assertEquals(conf.getPeriodicFlushFrequencyMilliSeconds(),
- DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT+1);
+ DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT + 1);
assertEquals(conf.getReaderIdleErrorThresholdMillis(),
DistributedLogConfiguration.BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT - 1);
}
@@ -73,7 +75,7 @@
DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
DistributedLogConfiguration override = new DistributedLogConfiguration();
override.setBKClientWriteTimeout(
- DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT+1);
+ DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT + 1);
conf.loadStreamConf(Optional.of(override));
assertEquals(conf.getBKClientWriteTimeout(),
DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
@@ -82,7 +84,7 @@
@Test(timeout = 20000)
public void loadStreamConfNullOverrides() throws Exception {
DistributedLogConfiguration conf = new DistributedLogConfiguration();
- DistributedLogConfiguration confClone = (DistributedLogConfiguration)conf.clone();
+ DistributedLogConfiguration confClone = (DistributedLogConfiguration) conf.clone();
Optional<DistributedLogConfiguration> streamConfiguration = Optional.absent();
conf.loadStreamConf(streamConfiguration);
@@ -105,7 +107,7 @@
@Test(timeout = 200000)
public void validateConfiguration(){
- boolean exceptionThrown=false;
+ boolean exceptionThrown = false;
DistributedLogConfiguration conf = new DistributedLogConfiguration();
// validate default configuration
conf.validate();
@@ -114,16 +116,16 @@
try {
conf.validate();
} catch (IllegalArgumentException e){
- exceptionThrown=true;
+ exceptionThrown = true;
}
assertFalse(exceptionThrown);
// test invalid case, should throw exception
- exceptionThrown=false;
+ exceptionThrown = false;
conf.setReadLACLongPollTimeout(conf.getBKClientReadTimeout() * 1000 * 2);
try {
conf.validate();
} catch (IllegalArgumentException e){
- exceptionThrown=true;
+ exceptionThrown = true;
}
assertTrue(exceptionThrown);
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java
index 5ea971e..fef3dd4 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java
@@ -32,16 +32,16 @@
import java.util.concurrent.CompletableFuture;
import org.apache.distributedlog.Entry.Reader;
import org.apache.distributedlog.Entry.Writer;
-import org.apache.distributedlog.exceptions.LogRecordTooLongException;
-import org.apache.distributedlog.io.CompressionCodec;
import org.apache.distributedlog.common.concurrent.FutureEventListener;
import org.apache.distributedlog.common.concurrent.FutureUtils;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.io.CompressionCodec;
import org.apache.distributedlog.util.Utils;
import org.junit.Assert;
import org.junit.Test;
/**
- * Test Case of {@link Entry}
+ * Test Case of {@link Entry}.
*/
public class TestEntry {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntryPosition.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntryPosition.java
index 60a0dab..6c6b0f5 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntryPosition.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntryPosition.java
@@ -17,12 +17,12 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
import org.junit.Test;
-import static org.junit.Assert.*;
/**
- * Test Case for {@link EntryPosition}
+ * Test Case for {@link EntryPosition}.
*/
public class TestEntryPosition {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
index c111baf..e59df36 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
@@ -17,6 +17,8 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogReader;
import org.apache.distributedlog.util.Utils;
@@ -24,9 +26,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+
+/**
+ * Test Cases for InterleavedReaders.
+ */
public class TestInterleavedReaders extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestInterleavedReaders.class);
@@ -86,8 +90,8 @@
Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
}
- Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
- Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+ Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
+ Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
if (null == reader0) {
reader0 = dlmreader0.getInputStream(1);
}
@@ -131,8 +135,8 @@
writer0.setForceRolling(false);
writer1.setForceRolling(false);
}
- Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
- Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+ Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
+ Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
LOG.info("Completed {} write", j);
if (null == reader0) {
reader0 = dlmreader0.getInputStream(1);
@@ -177,8 +181,8 @@
writer0.setForceRolling(false);
writer1.setForceRolling(false);
}
- Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
- Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+ Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
+ Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
if (null == reader0) {
reader0 = dlmreader0.getInputStream(1);
}
@@ -225,8 +229,8 @@
}
Thread.sleep(5);
}
- Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
- Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+ Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
+ Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
}
writer0.close();
writer1.close();
@@ -267,14 +271,14 @@
writer1.setForceRecovery(true);
}
DLSN dlsn1 = Utils.ioResult(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
- LOG.info("writer1 write record {} - txid = {}", dlsn1, txid-1);
+ LOG.info("writer1 write record {} - txid = {}", dlsn1, txid - 1);
DLSN dlsn0 = Utils.ioResult(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
- LOG.info("writer0 write record {} - txid = {}", dlsn0, txid-1);
+ LOG.info("writer0 write record {} - txid = {}", dlsn0, txid - 1);
writer0.setForceRecovery(false);
writer1.setForceRecovery(false);
}
- Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
- Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+ Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
+ Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
if (null == reader0) {
reader0 = dlmreader0.getInputStream(1);
}
@@ -320,8 +324,8 @@
writer0.setForceRolling(false);
writer1.setForceRolling(false);
}
- Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
- Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+ Utils.ioResult(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
+ Utils.ioResult(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid - 1)));
if (null == reader0) {
reader0 = dlmreader0.getInputStream(1);
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
index 8bdf86d..4f32127 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
@@ -17,9 +17,9 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
import java.net.URI;
import java.util.List;
-
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.namespace.Namespace;
import org.apache.distributedlog.api.namespace.NamespaceBuilder;
@@ -28,8 +28,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
+
+/**
+ * Test Cases for LogSegmentCreation.
+ */
public class TestLogSegmentCreation extends TestDistributedLogBase {
static Logger LOG = LoggerFactory.getLogger(TestLogSegmentCreation.class);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
index 39ffe85..3724b5a 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
@@ -17,6 +17,12 @@
*/
package org.apache.distributedlog;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import java.io.IOException;
import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataBuilder;
import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
import org.apache.distributedlog.LogSegmentMetadata.TruncationStatus;
@@ -28,15 +34,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
/**
- * Test {@link LogSegmentMetadata}
+ * Test {@link LogSegmentMetadata}.
*/
public class TestLogSegmentMetadata extends ZooKeeperClusterTestCase {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java
index fcc3395..e76ee6a 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java
@@ -17,15 +17,22 @@
*/
package org.apache.distributedlog;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Versioned;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.exceptions.DLIllegalStateException;
import org.apache.distributedlog.exceptions.UnexpectedException;
import org.apache.distributedlog.metadata.LogMetadata;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.util.DLUtils;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Versioned;
import org.apache.zookeeper.data.Stat;
import org.junit.Rule;
import org.junit.Test;
@@ -33,19 +40,18 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.net.URI;
-import java.util.List;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
+
+/**
+ * Test Cases for LogSegmentsZK.
+ */
public class TestLogSegmentsZK extends TestDistributedLogBase {
static Logger LOG = LoggerFactory.getLogger(TestLogSegmentsZK.class);
private static MaxLogSegmentSequenceNo getMaxLogSegmentSequenceNo(ZooKeeperClient zkc, URI uri, String streamName,
- DistributedLogConfiguration conf) throws Exception {
+ DistributedLogConfiguration conf) throws Exception {
Stat stat = new Stat();
String logSegmentsPath = LogMetadata.getLogSegmentsPath(
uri, streamName, conf.getUnpartitionedStreamName());
@@ -140,7 +146,8 @@
}
// invalid max ledger sequence number
- updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf, "invalid-max".getBytes(UTF_8));
+ updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace),
+ uri, streamName, conf, "invalid-max".getBytes(UTF_8));
DistributedLogManager dlm2 = namespace.openLog(streamName);
try {
dlm2.startLogSegmentNonPartitioned();
@@ -190,7 +197,7 @@
DistributedLogManager dlm1 = namespace.openLog(streamName);
try {
BKSyncLogWriter out1 = (BKSyncLogWriter) dlm1.startLogSegmentNonPartitioned();
- out1.write(DLMTestUtil.getLogRecordInstance(numSegments+1));
+ out1.write(DLMTestUtil.getLogRecordInstance(numSegments + 1));
out1.closeAndComplete();
fail("Should fail creating new log segment when encountered unmatch max ledger sequence number");
} catch (DLIllegalStateException lse) {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
index 2b02704..e15a40f 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
@@ -17,10 +17,12 @@
*/
package org.apache.distributedlog;
+
+import static org.apache.distributedlog.NonBlockingReadsTestUtil.*;
+import static org.junit.Assert.*;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
-
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogReader;
import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
@@ -32,11 +34,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.apache.distributedlog.NonBlockingReadsTestUtil.*;
-import static org.junit.Assert.*;
-
/**
- * {@link https://issues.apache.org/jira/browse/DL-12}
+ * {@link https://issues.apache.org/jira/browse/DL-12}.
*/
@DistributedLogAnnotations.FlakyTest
@Ignore
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
index 6c9e354..0fd1da4 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
@@ -17,7 +17,12 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
import com.google.common.util.concurrent.RateLimiter;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.distributedlog.api.AsyncLogWriter;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogReader;
@@ -25,13 +30,10 @@
import org.apache.distributedlog.util.Utils;
import org.junit.Test;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import static org.junit.Assert.*;
-
+/**
+ * Test Cases for NonBlockingReadsMultiReader.
+ */
public class TestNonBlockingReadsMultiReader extends TestDistributedLogBase {
static class ReaderThread extends Thread {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
index 1e8ed84..ba374bd 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
@@ -17,36 +17,35 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
import com.google.common.base.Optional;
import com.google.common.base.Ticker;
import com.google.common.collect.Lists;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.distributedlog.api.AsyncLogWriter;
import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
import org.apache.distributedlog.exceptions.DLIllegalStateException;
import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
import org.apache.distributedlog.injector.AsyncFailureInjector;
import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
import org.apache.distributedlog.util.ConfUtils;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Utils;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.NullStatsLogger;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.*;
/**
- * Test Case {@link ReadAheadEntryReader}
+ * Test Case {@link ReadAheadEntryReader}.
*/
public class TestReadAheadEntryReader extends TestDistributedLogBase {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java
index efc9ac6..bc81087 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java
@@ -17,27 +17,25 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
import org.apache.distributedlog.api.AsyncLogWriter;
import org.apache.distributedlog.logsegment.LogSegmentFilter;
import org.apache.distributedlog.util.Utils;
-
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
/**
- * Test {@link ReadUtils}
+ * Test {@link ReadUtils}.
*/
public class TestReadUtils extends TestDistributedLogBase {
@@ -59,15 +57,18 @@
);
}
- private CompletableFuture<LogRecordWithDLSN> getFirstGreaterThanRecord(BKDistributedLogManager bkdlm, int ledgerNo, DLSN dlsn) throws Exception {
+ private CompletableFuture<LogRecordWithDLSN> getFirstGreaterThanRecord(BKDistributedLogManager bkdlm,
+ int ledgerNo, DLSN dlsn) throws Exception {
List<LogSegmentMetadata> ledgerList = bkdlm.getLogSegments();
return ReadUtils.asyncReadFirstUserRecord(
- bkdlm.getStreamName(), ledgerList.get(ledgerNo), 2, 16, new AtomicInteger(0), Executors.newFixedThreadPool(1),
+ bkdlm.getStreamName(), ledgerList.get(ledgerNo),
+ 2, 16, new AtomicInteger(0), Executors.newFixedThreadPool(1),
bkdlm.getReaderEntryStore(), dlsn
);
}
- private CompletableFuture<LogRecordWithDLSN> getLastUserRecord(BKDistributedLogManager bkdlm, int ledgerNo) throws Exception {
+ private CompletableFuture<LogRecordWithDLSN> getLastUserRecord(BKDistributedLogManager bkdlm, int ledgerNo)
+ throws Exception {
BKLogReadHandler readHandler = bkdlm.createReadHandler();
List<LogSegmentMetadata> ledgerList = Utils.ioResult(
readHandler.readLogSegmentsFromStore(
@@ -76,7 +77,8 @@
null)
).getValue();
return ReadUtils.asyncReadLastRecord(
- bkdlm.getStreamName(), ledgerList.get(ledgerNo), false, false, false, 2, 16, new AtomicInteger(0), Executors.newFixedThreadPool(1),
+ bkdlm.getStreamName(), ledgerList.get(ledgerNo),
+ false, false, false, 2, 16, new AtomicInteger(0), Executors.newFixedThreadPool(1),
bkdlm.getReaderEntryStore()
);
}
@@ -87,7 +89,7 @@
BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1 /* txid */);
- DLSN dlsn = new DLSN(1,0,0);
+ DLSN dlsn = new DLSN(1, 0, 0);
CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
assertEquals("should be an exact match", dlsn, logrec.getDlsn());
@@ -100,7 +102,7 @@
BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1 /* txid */);
- DLSN dlsn = new DLSN(1,1,0);
+ DLSN dlsn = new DLSN(1, 1, 0);
CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
assertEquals("should be an exact match", dlsn, logrec.getDlsn());
@@ -113,10 +115,10 @@
BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1 /* txid */);
- DLSN dlsn = new DLSN(1,0,1);
+ DLSN dlsn = new DLSN(1, 0, 1);
CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
- assertEquals(new DLSN(1,1,0), logrec.getDlsn());
+ assertEquals(new DLSN(1, 1, 0), logrec.getDlsn());
bkdlm.close();
}
@@ -126,7 +128,7 @@
BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5 /* user recs */ , 1 /* txid */);
- DLSN dlsn = new DLSN(2,0,0);
+ DLSN dlsn = new DLSN(2, 0, 0);
CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
assertEquals(null, logrec);
@@ -142,10 +144,10 @@
txid += DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5 /* user recs */ , txid);
txid += DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5 /* user recs */ , txid);
- DLSN dlsn = new DLSN(1,3,0);
+ DLSN dlsn = new DLSN(1, 3, 0);
CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 1, dlsn);
LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
- assertEquals(new DLSN(2,0,0), logrec.getDlsn());
+ assertEquals(new DLSN(2, 0, 0), logrec.getDlsn());
bkdlm.close();
}
@@ -155,10 +157,10 @@
BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 5 /* control recs */, 5, 1 /* txid */);
- DLSN dlsn = new DLSN(1,3,0);
+ DLSN dlsn = new DLSN(1, 3, 0);
CompletableFuture<LogRecordWithDLSN> futureLogrec = getFirstGreaterThanRecord(bkdlm, 0, dlsn);
LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
- assertEquals(new DLSN(1,5,0), logrec.getDlsn());
+ assertEquals(new DLSN(1, 5, 0), logrec.getDlsn());
bkdlm.close();
}
@@ -170,7 +172,7 @@
CompletableFuture<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
- assertEquals(new DLSN(1,9,0), logrec.getDlsn());
+ assertEquals(new DLSN(1, 9, 0), logrec.getDlsn());
bkdlm.close();
}
@@ -190,7 +192,7 @@
CompletableFuture<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
LogRecordWithDLSN logrec = Utils.ioResult(futureLogrec);
- assertEquals(new DLSN(1,2,0), logrec.getDlsn());
+ assertEquals(new DLSN(1, 2, 0), logrec.getDlsn());
bkdlm.close();
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReader.java
index 8d9f846..44ae00f 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReader.java
@@ -17,7 +17,15 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
+import java.io.IOException;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.distributedlog.api.AsyncLogReader;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.common.concurrent.FutureEventListener;
@@ -25,15 +33,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import static org.junit.Assert.*;
/**
* A Reader wraps reading next logic for testing.
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
index 7111e07..3eb82c9 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
@@ -17,33 +17,37 @@
*/
package org.apache.distributedlog;
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
-
-import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.api.LogReader;
-import org.apache.distributedlog.feature.CoreFeatureKeys;
-import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryReader;
-import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.util.Utils;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.distributedlog.api.DistributedLogManager;
+import org.apache.distributedlog.api.LogReader;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations.FlakyTest;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.feature.CoreFeatureKeys;
+import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryReader;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.util.Utils;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.distributedlog.common.annotations.DistributedLogAnnotations.FlakyTest;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
+
+
+/**
+ * Test Cases for RollLogSegments.
+ */
public class TestRollLogSegments extends TestDistributedLogBase {
- static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class);
+ private static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class);
private static void ensureOnlyOneInprogressLogSegments(List<LogSegmentMetadata> segments) throws Exception {
int numInprogress = 0;
@@ -268,7 +272,8 @@
// send requests in parallel to have outstanding requests
for (int i = 1; i <= numLogSegments; i++) {
final int entryId = i;
- CompletableFuture<DLSN> writeFuture = writer.write(DLMTestUtil.getLogRecordInstance(entryId)).whenComplete(new FutureEventListener<DLSN>() {
+ CompletableFuture<DLSN> writeFuture = writer.write(DLMTestUtil.getLogRecordInstance(entryId))
+ .whenComplete(new FutureEventListener<DLSN>() {
@Override
public void onSuccess(DLSN value) {
logger.info("Completed entry {} : {}.", entryId, value);
@@ -405,7 +410,8 @@
// simulate a recovery without closing ledger causing recording wrong last dlsn
BKLogWriteHandler writeHandler = writer.getCachedWriteHandler();
writeHandler.completeAndCloseLogSegment(
- writeHandler.inprogressZNodeName(perStreamWriter.getLogSegmentId(), perStreamWriter.getStartTxId(), perStreamWriter.getLogSegmentSequenceNumber()),
+ writeHandler.inprogressZNodeName(perStreamWriter.getLogSegmentId(),
+ perStreamWriter.getStartTxId(), perStreamWriter.getLogSegmentSequenceNumber()),
perStreamWriter.getLogSegmentSequenceNumber(),
perStreamWriter.getLogSegmentId(),
perStreamWriter.getStartTxId(), perStreamWriter.getLastTxId(),
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java
index da4ef81..e45f14c 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java
@@ -17,6 +17,9 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
import org.apache.distributedlog.api.AsyncLogReader;
import org.apache.distributedlog.common.concurrent.FutureEventListener;
@@ -25,17 +28,13 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.List;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import static org.junit.Assert.*;
/**
* Test Cases related to sequence ids.
*/
public class TestSequenceID extends TestDistributedLogBase {
- static final Logger logger = LoggerFactory.getLogger(TestSequenceID.class);
+ private static final Logger logger = LoggerFactory.getLogger(TestSequenceID.class);
@Test(timeout = 60000)
public void testCompleteV4LogSegmentAsV4() throws Exception {
@@ -214,7 +213,7 @@
List<LogSegmentMetadata> segmentsv4 = dlmv4.getLogSegments();
assertEquals(11, segmentsv4.size());
- for(int i = 7; i < 11; i++) {
+ for (int i = 7; i < 11; i++) {
assertFalse(segmentsv4.get(i).isInProgress());
assertTrue(segmentsv4.get(i).getStartSequenceId() < 0);
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java
index 06708c8..ae2ebf4 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java
@@ -17,25 +17,29 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
import java.net.URI;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.distributedlog.LogSegmentMetadata.TruncationStatus;
import org.apache.distributedlog.api.AsyncLogWriter;
import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogReader;
import org.apache.distributedlog.util.Utils;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
+
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.distributedlog.LogSegmentMetadata.TruncationStatus;
-import static org.junit.Assert.*;
+
+/**
+ * Test Cases for truncation.
+ */
public class TestTruncate extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestTruncate.class);
@@ -45,7 +49,8 @@
.setOutputBufferSize(0)
.setPeriodicFlushFrequencyMilliSeconds(10)
.setSchedulerShutdownTimeoutMs(0)
- .setDLLedgerMetadataLayoutVersion(LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value);
+ .setDLLedgerMetadataLayoutVersion(
+ LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value);
static void updateCompletionTime(ZooKeeperClient zkc,
LogSegmentMetadata l, long completionTime) throws Exception {
@@ -78,7 +83,7 @@
.build();
// Update completion time of first 5 segments
- long newTimeMs = System.currentTimeMillis() - 60*60*1000*2;
+ long newTimeMs = System.currentTimeMillis() - 60 * 60 * 1000 * 2;
for (int i = 0; i < 5; i++) {
LogSegmentMetadata segment = segments.get(i);
updateCompletionTime(zkc, segment, newTimeMs + i);
@@ -128,8 +133,8 @@
verifyEntries(name, 1, 1, 5 * 10);
for (int i = 1; i <= 4; i++) {
- int txn = (i-1) * 10 + i;
- DLSN dlsn = txid2DLSN.get((long)txn);
+ int txn = (i - 1) * 10 + i;
+ DLSN dlsn = txid2DLSN.get((long) txn);
assertTrue(Utils.ioResult(pair.getRight().truncate(dlsn)));
verifyEntries(name, 1, (i - 1) * 10 + 1, (5 - i + 1) * 10);
}
@@ -159,8 +164,8 @@
Thread.sleep(1000);
for (int i = 1; i <= 4; i++) {
- int txn = (i-1) * 10 + i;
- DLSN dlsn = txid2DLSN.get((long)txn);
+ int txn = (i - 1) * 10 + i;
+ DLSN dlsn = txid2DLSN.get((long) txn);
assertTrue(Utils.ioResult(pair.getRight().truncate(dlsn)));
verifyEntries(name, 1, (i - 1) * 10 + 1, (5 - i + 1) * 10);
}
@@ -175,7 +180,7 @@
pair.getLeft().close();
// Try force truncation
- BKDistributedLogManager dlm = (BKDistributedLogManager)createNewDLM(confLocal, name);
+ BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
BKLogWriteHandler handler = dlm.createWriteHandler(true);
Utils.ioResult(handler.purgeLogSegmentsOlderThanTxnId(Integer.MAX_VALUE));
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
index 0d0ca99..9a56edd 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
@@ -17,19 +17,21 @@
*/
package org.apache.distributedlog;
-import org.apache.distributedlog.exceptions.OverCapacityException;
-import org.apache.distributedlog.util.SimplePermitLimiter;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
import org.apache.bookkeeper.feature.Feature;
import org.apache.bookkeeper.feature.SettableFeature;
import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.util.SimplePermitLimiter;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-
+/**
+ * Test Cases for {@link org.apache.distributedlog.util.SimplePermitLimiter}.
+ */
public class TestWriteLimiter {
static final Logger LOG = LoggerFactory.getLogger(TestWriteLimiter.class);
@@ -196,7 +198,8 @@
assertPermits(streamLimiter, 0, globalLimiter, 0);
}
- void assertPermits(SimplePermitLimiter streamLimiter, int streamPermits, SimplePermitLimiter globalLimiter, int globalPermits) {
+ void assertPermits(SimplePermitLimiter streamLimiter,
+ int streamPermits, SimplePermitLimiter globalLimiter, int globalPermits) {
assertEquals(streamPermits, streamLimiter.getPermits());
assertEquals(globalPermits, globalLimiter.getPermits());
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
index 75bcda2..777e9ec 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
@@ -17,11 +17,20 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
import org.apache.distributedlog.ZooKeeperClient.Credentials;
import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
@@ -38,23 +47,15 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-import static org.junit.Assert.*;
/**
- * Test Cases for {@link org.apache.distributedlog.ZooKeeperClient}
+ * Test Cases for {@link org.apache.distributedlog.ZooKeeperClient}.
*/
public class TestZooKeeperClient extends ZooKeeperClusterTestCase {
static final Logger LOG = LoggerFactory.getLogger(TestZooKeeperClient.class);
- private final static int sessionTimeoutMs = 2000;
+ private static final int sessionTimeoutMs = 2000;
private ZooKeeperClient zkc;
@@ -104,7 +105,8 @@
ZooKeeperClient zkcAuth = buildAuthdClient("test");
zkcAuth.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
zkcAuth.get().create("/test/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- zkcAuth.get().create("/test/key2", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test/key2", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
ZooKeeperClient zkcNoAuth = buildClient();
zkcNoAuth.get().create("/test/key1/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
@@ -124,7 +126,8 @@
zkcAuth.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
zkcAuth.get().create("/test/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
try {
- zkcAuth.get().create("/test/key2", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test/key2", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
fail("create should fail because we're not authenticated");
} catch (KeeperException.InvalidACLException ex) {
LOG.info("caught exception writing to protected key", ex);
@@ -136,9 +139,12 @@
@Test(timeout = 60000)
public void testAclAllowsReadsForNoAuth() throws Exception {
ZooKeeperClient zkcAuth = buildAuthdClient("test");
- zkcAuth.get().create("/test", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
- zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
- zkcAuth.get().create("/test/key1/key2", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test/key1", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test/key1/key2", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
ZooKeeperClient zkcNoAuth = buildClient();
List<String> nodes = null;
@@ -164,7 +170,8 @@
zkcAuth.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
try {
- zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test/key1", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
fail("should have failed");
} catch (Exception ex) {
}
@@ -173,7 +180,8 @@
credentials.authenticate(zkcAuth.get());
// Should not throw now that we're authenticated.
- zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test/key1", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
rmAll(zkcAuth, "/test");
}
@@ -252,14 +260,15 @@
}
/**
- * {@link https://issues.apache.org/jira/browse/DL-34}
+ * {@link https://issues.apache.org/jira/browse/DL-34}.
*/
@DistributedLogAnnotations.FlakyTest
@Ignore
@Test(timeout = 60000)
public void testAclAuthSpansExpiration() throws Exception {
ZooKeeperClient zkcAuth = buildAuthdClient("test");
- zkcAuth.get().create("/test", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
CountDownLatch expired = awaitConnectionEvent(KeeperState.Expired, zkcAuth);
CountDownLatch connected = awaitConnectionEvent(KeeperState.SyncConnected, zkcAuth);
@@ -269,20 +278,22 @@
expired.await(2, TimeUnit.SECONDS);
connected.await(2, TimeUnit.SECONDS);
- zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test/key1", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
rmAll(zkcAuth, "/test");
}
/**
- * {@link https://issues.apache.org/jira/browse/DL-34}
+ * {@link https://issues.apache.org/jira/browse/DL-34}.
*/
@DistributedLogAnnotations.FlakyTest
@Ignore
@Test(timeout = 60000)
public void testAclAuthSpansExpirationNonRetryableClient() throws Exception {
ZooKeeperClient zkcAuth = clientBuilder().retryPolicy(null).zkAclId("test").build();
- zkcAuth.get().create("/test", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
CountDownLatch expired = awaitConnectionEvent(KeeperState.Expired, zkcAuth);
CountDownLatch connected = awaitConnectionEvent(KeeperState.SyncConnected, zkcAuth);
@@ -292,7 +303,8 @@
expired.await(2, TimeUnit.SECONDS);
connected.await(2, TimeUnit.SECONDS);
- zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+ zkcAuth.get().create("/test/key1", new byte[0],
+ DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
rmAll(zkcAuth, "/test");
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClientBuilder.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClientBuilder.java
index 9fa0279..6f6f348 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClientBuilder.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClientBuilder.java
@@ -17,8 +17,8 @@
*/
package org.apache.distributedlog;
-import org.apache.distributedlog.util.RetryPolicyUtils;
import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.distributedlog.util.RetryPolicyUtils;
/**
* The zookeeper client builder used for testing.
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java
index cafdcd6..cbf02b8 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java
@@ -17,8 +17,10 @@
*/
package org.apache.distributedlog;
+import static org.junit.Assert.*;
import com.google.common.base.Stopwatch;
-
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
@@ -26,17 +28,14 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import static org.junit.Assert.*;
/**
- * Utilities of {@link org.apache.distributedlog.ZooKeeperClient}
+ * Utilities of {@link org.apache.distributedlog.ZooKeeperClient}.
*/
public class ZooKeeperClientUtils {
- static final Logger logger = LoggerFactory.getLogger(ZooKeeperClientUtils.class);
+ private static final Logger logger = LoggerFactory.getLogger(ZooKeeperClientUtils.class);
/**
* Expire given zookeeper client's session.
@@ -58,8 +57,8 @@
@Override
public void process(WatchedEvent event) {
logger.debug("Receive event : {}", event);
- if (event.getType() == Event.EventType.None &&
- event.getState() == Event.KeeperState.Expired) {
+ if (event.getType() == Event.EventType.None
+ && event.getState() == Event.KeeperState.Expired) {
expireLatch.countDown();
}
}
@@ -67,8 +66,8 @@
ZooKeeper newZk = new ZooKeeper(zkServers, timeout, new Watcher() {
@Override
public void process(WatchedEvent event) {
- if (Event.EventType.None == event.getType() &&
- Event.KeeperState.SyncConnected == event.getState()) {
+ if (Event.EventType.None == event.getType()
+ && Event.KeeperState.SyncConnected == event.getState()) {
latch.countDown();
}
}
@@ -80,7 +79,7 @@
boolean done = false;
Stopwatch expireWait = Stopwatch.createStarted();
- while (!done && expireWait.elapsed(TimeUnit.MILLISECONDS) < timeout*2) {
+ while (!done && expireWait.elapsed(TimeUnit.MILLISECONDS) < timeout * 2) {
try {
zkc.get().exists("/", false);
done = true;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java b/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java
index 2b6204d..1f7ac60 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java
@@ -17,16 +17,18 @@
*/
package org.apache.distributedlog;
+import java.io.File;
import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
import org.apache.bookkeeper.util.IOUtils;
-import org.apache.bookkeeper.util.LocalBookKeeper;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.junit.AfterClass;
import org.junit.BeforeClass;
-import java.io.File;
+/**
+ * ZooKeeperClusterTestCase.
+ */
public class ZooKeeperClusterTestCase {
protected static File zkDir;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
index 45fc1f3..3275b84 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
@@ -17,13 +17,15 @@
*/
package org.apache.distributedlog.acl;
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
import java.net.URI;
import org.apache.distributedlog.TestZooKeeperClientBuilder;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.ZooKeeperClusterTestCase;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.impl.acl.ZKAccessControl;
import org.apache.distributedlog.thrift.AccessControlEntry;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -31,9 +33,12 @@
import org.junit.Before;
import org.junit.Test;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
+
+
+/**
+ * TestZKAccessControl.
+ */
public class TestZKAccessControl extends ZooKeeperClusterTestCase {
private ZooKeeperClient zkc;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
index 868549e..56cfd76 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
@@ -17,6 +17,10 @@
*/
package org.apache.distributedlog.acl;
+import static org.junit.Assert.*;
+import java.net.URI;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.TestZooKeeperClientBuilder;
import org.apache.distributedlog.ZooKeeperClient;
@@ -32,15 +36,13 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.net.URI;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import static org.junit.Assert.*;
-
+/**
+ * TestZKAccessControlManager.
+ */
public class TestZKAccessControlManager extends ZooKeeperClusterTestCase {
- static final Logger logger = LoggerFactory.getLogger(TestZKAccessControlManager.class);
+ private static final Logger logger = LoggerFactory.getLogger(TestZKAccessControlManager.class);
private DistributedLogConfiguration conf;
private ZooKeeperClient zkc;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
index 8a2c476..e42b4ea 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
@@ -17,20 +17,29 @@
*/
package org.apache.distributedlog.admin;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.LogSegmentMetadata;
import org.apache.distributedlog.TestDistributedLogBase;
import org.apache.distributedlog.TestZooKeeperClientBuilder;
import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.common.util.SchedulerUtils;
import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.common.util.SchedulerUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs;
import org.junit.After;
@@ -39,17 +48,13 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
+
+
+/**
+ * TestDLCK.
+ */
public class TestDLCK extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestDLCK.class);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
index f7f859c..1174b56 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
@@ -17,15 +17,26 @@
*/
package org.apache.distributedlog.admin;
+import static org.junit.Assert.*;
+
import java.net.URI;
import java.util.concurrent.CompletableFuture;
-
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.TestDistributedLogBase;
import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.api.AsyncLogReader;
+import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
+import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
import org.apache.distributedlog.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs;
@@ -36,19 +47,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.distributedlog.api.AsyncLogReader;
-import org.apache.distributedlog.DLMTestUtil;
-import org.apache.distributedlog.DLSN;
-import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.LogRecord;
-import org.apache.distributedlog.LogRecordWithDLSN;
-import org.apache.distributedlog.TestDistributedLogBase;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
-import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import static org.junit.Assert.*;
-
+/**
+ * TestDistributedLogAdmin.
+ */
public class TestDistributedLogAdmin extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogAdmin.class);
@@ -70,7 +72,7 @@
}
/**
- * {@link https://issues.apache.org/jira/browse/DL-44}
+ * {@link https://issues.apache.org/jira/browse/DL-44}.
*/
@DistributedLogAnnotations.FlakyTest
@Ignore
@@ -144,7 +146,8 @@
// Dryrun
DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
- new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
+ new DryrunLogSegmentMetadataStoreUpdater(confLocal,
+ getLogSegmentMetadataStore(namespace)), streamName, false, false);
try {
reader = readDLM.getAsyncLogReader(lastDLSN);
@@ -158,7 +161,8 @@
// Actual run
DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
- LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
+ LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal,
+ getLogSegmentMetadataStore(namespace)), streamName, false, false);
// be able to read more after fix
reader = readDLM.getAsyncLogReader(lastDLSN);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java
index 925cad5..f146c1e 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java
@@ -17,27 +17,39 @@
*/
package org.apache.distributedlog.bk;
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import java.net.URI;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Set;
import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.BookKeeperClient;
-import org.apache.distributedlog.BookKeeperClientBuilder;
-import org.apache.distributedlog.TestZooKeeperClientBuilder;
-import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
-import org.apache.distributedlog.bk.SimpleLedgerAllocator.AllocationException;
-import org.apache.distributedlog.bk.SimpleLedgerAllocator.Phase;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.TestDistributedLogBase;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.util.Transaction.OpListener;
-import org.apache.distributedlog.util.Utils;
-import org.apache.distributedlog.zk.DefaultZKOp;
-import org.apache.distributedlog.zk.ZKTransaction;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.meta.ZkVersion;
import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.BookKeeperClientBuilder;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.bk.SimpleLedgerAllocator.AllocationException;
+import org.apache.distributedlog.bk.SimpleLedgerAllocator.Phase;
+import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
+
+
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.util.Transaction.OpListener;
+import org.apache.distributedlog.util.Utils;
+import org.apache.distributedlog.zk.DefaultZKOp;
+import org.apache.distributedlog.zk.ZKTransaction;
+
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
@@ -52,16 +64,12 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.net.URI;
-import java.util.Enumeration;
-import java.util.HashSet;
-import java.util.Set;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+
+/**
+ * TestLedgerAllocator.
+ */
public class TestLedgerAllocator extends TestDistributedLogBase {
private static final Logger logger = LoggerFactory.getLogger(TestLedgerAllocator.class);
@@ -124,7 +132,7 @@
}
/**
- * {@link https://issues.apache.org/jira/browse/DL-43}
+ * {@link https://issues.apache.org/jira/browse/DL-43}.
*/
@DistributedLogAnnotations.FlakyTest
@Ignore
@@ -190,7 +198,8 @@
long eid = lh.addEntry("hello world".getBytes());
lh.close();
- LedgerHandle readLh = bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
+ LedgerHandle readLh = bkc.get().openLedger(lh.getId(),
+ BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
Enumeration<LedgerEntry> entries = readLh.readEntries(eid, eid);
int i = 0;
while (entries.hasMoreElements()) {
@@ -255,7 +264,8 @@
// should fail to commit txn1 as version is changed by second allocator
try {
Utils.ioResult(txn1.execute());
- fail("Should fail commit obtaining ledger handle from first allocator as allocator is modified by second allocator.");
+ fail("Should fail commit obtaining ledger handle from first allocator"
+ + " as allocator is modified by second allocator.");
} catch (ZKException ke) {
// as expected
}
@@ -278,7 +288,8 @@
}
long eid = lh2.addEntry("hello world".getBytes());
lh2.close();
- LedgerHandle readLh = bkc.get().openLedger(lh2.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
+ LedgerHandle readLh = bkc.get().openLedger(lh2.getId(),
+ BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
Enumeration<LedgerEntry> entries = readLh.readEntries(eid, eid);
int i = 0;
while (entries.hasMoreElements()) {
@@ -306,7 +317,7 @@
}
/**
- * {@link https://issues.apache.org/jira/browse/DL-26}
+ * {@link https://issues.apache.org/jira/browse/DL-26}.
*/
@DistributedLogAnnotations.FlakyTest
@Ignore
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
index a42d688..f82129d 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
@@ -17,25 +17,9 @@
*/
package org.apache.distributedlog.bk;
+import static org.junit.Assert.*;
+
import com.google.common.collect.Lists;
-import org.apache.distributedlog.BookKeeperClient;
-import org.apache.distributedlog.BookKeeperClientBuilder;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.TestDistributedLogBase;
-import org.apache.distributedlog.TestZooKeeperClientBuilder;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.util.Transaction.OpListener;
-import org.apache.distributedlog.util.Utils;
-import org.apache.distributedlog.zk.ZKTransaction;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.zookeeper.data.Stat;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URI;
@@ -47,9 +31,31 @@
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.BookKeeperClientBuilder;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.util.Transaction.OpListener;
+import org.apache.distributedlog.util.Utils;
+import org.apache.distributedlog.zk.ZKTransaction;
+import org.apache.zookeeper.data.Stat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
+
+
+/**
+ * TestLedgerAllocatorPool.
+ */
public class TestLedgerAllocatorPool extends TestDistributedLogBase {
private static final Logger logger = LoggerFactory.getLogger(TestLedgerAllocatorPool.class);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java
index 21aa1c9..caae387 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.java
@@ -17,16 +17,14 @@
*/
package org.apache.distributedlog.config;
+import static org.junit.Assert.*;
import com.google.common.base.Objects;
import com.google.common.base.Optional;
-
-import org.apache.distributedlog.DistributedLogConfiguration;
-
import java.io.File;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
-
+import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
import org.apache.distributedlog.common.config.PropertiesWriter;
@@ -34,8 +32,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
+/**
+ * TestDynamicConfigurationFactory.
+ */
public class TestDynamicConfigurationFactory {
static final Logger LOG = LoggerFactory.getLogger(TestDynamicConfigurationFactory.class);
@@ -62,7 +62,8 @@
PropertiesWriter writer = new PropertiesWriter();
DynamicConfigurationFactory factory = getConfigFactory(writer.getFile());
Optional<DynamicDistributedLogConfiguration> conf = factory.getDynamicConfiguration(writer.getFile().getPath());
- assertEquals(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT, conf.get().getRetentionPeriodHours());
+ assertEquals(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT,
+ conf.get().getRetentionPeriodHours());
writer.setProperty(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS, "1");
writer.save();
waitForConfig(conf.get(), 1);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
index 2731af3..993182c 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
@@ -17,16 +17,19 @@
*/
package org.apache.distributedlog.config;
-import org.apache.distributedlog.DistributedLogConfiguration;
+import static org.apache.distributedlog.DistributedLogConfiguration.*;
+import static org.junit.Assert.*;
+import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.bk.QuorumConfig;
import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
import org.apache.distributedlog.common.config.ConcurrentConstConfiguration;
import org.junit.Test;
-import static org.apache.distributedlog.DistributedLogConfiguration.*;
-import static org.junit.Assert.*;
+/**
+ * TestDynamicDistributedLogConfiguration.
+ */
public class TestDynamicDistributedLogConfiguration {
@Test(timeout = 20000)
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
index 1064a6f..359484b 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
@@ -17,18 +17,18 @@
*/
package org.apache.distributedlog.feature;
-import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.junit.Test;
-
+import static org.junit.Assert.*;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.distributedlog.common.config.ConcurrentBaseConfiguration;
+import org.junit.Test;
-import static org.junit.Assert.*;
+
/**
- * Test case for configuration based feature provider
+ * Test case for configuration based feature provider.
*/
public class TestConfigurationFeatureProvider {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
index f8dd245..d42d3ec 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
@@ -17,25 +17,25 @@
*/
package org.apache.distributedlog.feature;
+import static org.junit.Assert.*;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.common.annotations.DistributedLogAnnotations;
import org.apache.distributedlog.common.config.PropertiesWriter;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.NullStatsLogger;
import org.junit.Ignore;
import org.junit.Test;
-import static org.junit.Assert.*;
+
/**
- * Test case for dynamic configuration based feature provider
+ * Test case for dynamic configuration based feature provider.
*/
public class TestDynamicConfigurationFeatureProvider {
/**
- * Make sure config is reloaded
- *
- * Give FileChangedReloadingStrategy some time to allow reloading
+ * Make sure config is reloaded.
+ *Give FileChangedReloadingStrategy some time to allow reloading
* Make sure now!=lastChecked
* {@link org.apache.commons.configuration.reloading.FileChangedReloadingStrategy#reloadingRequired()}
*/
@@ -77,7 +77,7 @@
}
/**
- * {@link https://issues.apache.org/jira/browse/DL-40}
+ * {@link https://issues.apache.org/jira/browse/DL-40}.
*/
@DistributedLogAnnotations.FlakyTest
@Ignore
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
index db9fb31..93e2802 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
@@ -17,8 +17,11 @@
*/
package org.apache.distributedlog.impl;
+import static org.junit.Assert.*;
import com.google.common.base.Optional;
import com.google.common.collect.Sets;
+import java.net.URI;
+import java.util.Set;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.TestDistributedLogBase;
import org.apache.distributedlog.TestZooKeeperClientBuilder;
@@ -33,17 +36,14 @@
import org.junit.Test;
import org.junit.rules.TestName;
-import java.net.URI;
-import java.util.Set;
-import static org.junit.Assert.*;
/**
* Test ZK based metadata store.
*/
public class TestZKLogMetadataStore extends TestDistributedLogBase {
- private final static int zkSessionTimeoutMs = 2000;
+ private static final int zkSessionTimeoutMs = 2000;
@Rule
public TestName runtime = new TestName();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java
index 29a9add..eac11a9 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java
@@ -17,22 +17,27 @@
*/
package org.apache.distributedlog.impl;
+import static org.apache.distributedlog.impl.ZKLogSegmentFilters.*;
+import static org.junit.Assert.*;
import com.google.common.collect.Sets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.DistributedLogConstants;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import static org.apache.distributedlog.impl.ZKLogSegmentFilters.*;
-import static org.junit.Assert.*;
+
+
+/**
+ * TestZKLogSegmentFilters.
+ */
public class TestZKLogSegmentFilters {
static final Logger LOG = LoggerFactory.getLogger(TestZKLogSegmentFilters.class);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
index 721bfbf..6e1d0b8 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
@@ -17,8 +17,19 @@
*/
package org.apache.distributedlog.impl;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
import com.google.common.collect.Lists;
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.LogSegmentMetadata;
@@ -27,16 +38,14 @@
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.ZooKeeperClientUtils;
import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.exceptions.ZKException;
import org.apache.distributedlog.metadata.LogMetadata;
import org.apache.distributedlog.metadata.LogMetadataForWriter;
import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.common.concurrent.FutureUtils;
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Transaction;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
+
import org.apache.distributedlog.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -50,15 +59,9 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.net.URI;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+
+
/**
* Test ZK based log segment metadata store.
@@ -67,7 +70,7 @@
private static final Logger logger = LoggerFactory.getLogger(TestZKLogSegmentMetadataStore.class);
- private final static int zkSessionTimeoutMs = 2000;
+ private static final int zkSessionTimeoutMs = 2000;
private LogSegmentMetadata createLogSegment(
long logSegmentSequenceNumber) {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
index 4bd513b..1082d7e 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
@@ -17,7 +17,15 @@
*/
package org.apache.distributedlog.impl;
+import static org.junit.Assert.*;
import com.google.common.collect.Sets;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.TestDistributedLogBase;
import org.apache.distributedlog.TestZooKeeperClientBuilder;
@@ -34,21 +42,14 @@
import org.junit.Test;
import org.junit.rules.TestName;
-import java.net.URI;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import static org.junit.Assert.*;
/**
* Test ZK Namespace Watcher.
*/
public class TestZKNamespaceWatcher extends TestDistributedLogBase {
- private final static int zkSessionTimeoutMs = 2000;
+ private static final int zkSessionTimeoutMs = 2000;
@Rule
public TestName runtime = new TestName();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
index c81eb1d..190c9d9 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
@@ -17,10 +17,23 @@
*/
package org.apache.distributedlog.impl.federated;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import com.google.common.base.Optional;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.TestDistributedLogBase;
import org.apache.distributedlog.TestZooKeeperClientBuilder;
@@ -39,28 +52,16 @@
import org.junit.Test;
import org.junit.rules.TestName;
-import java.net.URI;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+
/**
* Test ZK based metadata store.
*/
public class TestFederatedZKLogMetadataStore extends TestDistributedLogBase {
- private final static int zkSessionTimeoutMs = 2000;
- private final static int maxLogsPerSubnamespace = 10;
+ private static final int zkSessionTimeoutMs = 2000;
+ private static final int maxLogsPerSubnamespace = 10;
static class TestNamespaceListener implements NamespaceListener {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
index a70edf5..f8848c2 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
@@ -17,15 +17,17 @@
*/
package org.apache.distributedlog.impl.logsegment;
+import static org.junit.Assert.*;
import com.google.common.collect.Lists;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.api.AsyncLogWriter;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.distributedlog.BookKeeperClient;
import org.apache.distributedlog.BookKeeperClientBuilder;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.Entry;
import org.apache.distributedlog.LogRecord;
import org.apache.distributedlog.LogRecordWithDLSN;
@@ -33,6 +35,8 @@
import org.apache.distributedlog.TestDistributedLogBase;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.api.AsyncLogWriter;
+import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.exceptions.EndOfLogSegmentException;
import org.apache.distributedlog.exceptions.ReadCancelledException;
import org.apache.distributedlog.injector.AsyncFailureInjector;
@@ -40,20 +44,17 @@
import org.apache.distributedlog.util.ConfUtils;
import org.apache.distributedlog.util.OrderedScheduler;
import org.apache.distributedlog.util.Utils;
-import org.apache.bookkeeper.stats.NullStatsLogger;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestName;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import static org.junit.Assert.*;
+
/**
- * Test Case for {@link BKLogSegmentEntryReader}
+ * Test Case for {@link BKLogSegmentEntryReader}.
*/
public class TestBKLogSegmentEntryReader extends TestDistributedLogBase {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
index 813501b..cd894ae 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
@@ -17,24 +17,31 @@
*/
package org.apache.distributedlog.impl.metadata;
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.*;
+import static org.apache.distributedlog.metadata.LogMetadata.*;
+import static org.junit.Assert.*;
import com.google.common.collect.Lists;
-import org.apache.distributedlog.DLMTestUtil;
-import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.api.MetadataAccessor;
-import org.apache.distributedlog.TestZooKeeperClientBuilder;
-import org.apache.distributedlog.api.namespace.Namespace;
-import org.apache.distributedlog.metadata.DLMetadata;
-import org.apache.distributedlog.metadata.LogMetadataForWriter;
-import org.apache.distributedlog.api.namespace.NamespaceBuilder;
-import org.apache.distributedlog.DistributedLogConstants;
-import org.apache.distributedlog.exceptions.LogNotFoundException;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.ZooKeeperClusterTestCase;
-import org.apache.distributedlog.util.DLUtils;
-import org.apache.distributedlog.util.Utils;
+import java.net.URI;
+import java.util.List;
import org.apache.bookkeeper.meta.ZkVersion;
import org.apache.bookkeeper.util.ZkUtils;
import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClusterTestCase;
+import org.apache.distributedlog.api.MetadataAccessor;
+import org.apache.distributedlog.api.namespace.Namespace;
+import org.apache.distributedlog.api.namespace.NamespaceBuilder;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.distributedlog.metadata.LogMetadataForWriter;
+
+import org.apache.distributedlog.util.DLUtils;
+import org.apache.distributedlog.util.Utils;
+
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Transaction;
@@ -47,21 +54,18 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.net.URI;
-import java.util.List;
-import static org.apache.distributedlog.metadata.LogMetadata.*;
-import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.*;
-import static org.junit.Assert.*;
+
+
/**
- * Test {@link ZKLogStreamMetadataStore}
+ * Test {@link ZKLogStreamMetadataStore}.
*/
public class TestZKLogStreamMetadataStore extends ZooKeeperClusterTestCase {
private static final Logger logger = LoggerFactory.getLogger(TestZKLogStreamMetadataStore.class);
- private final static int sessionTimeoutMs = 30000;
+ private static final int sessionTimeoutMs = 30000;
@Rule
public TestName testName = new TestName();
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java
index 3631ae9..bc45b9c 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java
@@ -17,22 +17,26 @@
*/
package org.apache.distributedlog.impl.metadata;
+
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.*;
+import static org.junit.Assert.*;
import com.google.common.collect.Lists;
+import java.net.URI;
+import java.util.List;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Versioned;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.exceptions.UnexpectedException;
import org.apache.distributedlog.metadata.LogMetadata;
import org.apache.distributedlog.metadata.LogMetadataForWriter;
import org.apache.distributedlog.util.DLUtils;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Versioned;
+
import org.junit.Test;
-import java.net.URI;
-import java.util.List;
-import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.*;
-import static org.junit.Assert.*;
-
+/**
+ * TestZKLogStreamMetadataStoreUtils.
+ */
public class TestZKLogStreamMetadataStoreUtils {
@SuppressWarnings("unchecked")
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java
index 24ff0da..1c577f6 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java
@@ -17,30 +17,35 @@
*/
package org.apache.distributedlog.impl.metadata;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import java.io.IOException;
+import java.net.URI;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.DistributedLogConstants;
import org.apache.distributedlog.TestZooKeeperClientBuilder;
-import org.apache.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.distributedlog.impl.metadata.ZkMetadataResolver;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClusterTestCase;
+
import org.apache.distributedlog.metadata.DLMetadata;
import org.apache.distributedlog.util.Utils;
-import org.apache.distributedlog.ZooKeeperClient;
-import org.apache.distributedlog.ZooKeeperClientBuilder;
-import org.apache.distributedlog.ZooKeeperClusterTestCase;
+
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
-import java.io.IOException;
-import java.net.URI;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+
+
+
+/**
+ * TestZkMetadataResolver.
+ */
public class TestZkMetadataResolver extends ZooKeeperClusterTestCase {
private static final BKDLConfig bkdlConfig = new BKDLConfig("127.0.0.1:7000", "ledgers");
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java
index c368ca8..5861718 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java
@@ -17,10 +17,14 @@
*/
package org.apache.distributedlog.limiter;
+import static org.junit.Assert.*;
import org.junit.Test;
-import static org.junit.Assert.*;
+
+/**
+ * TestRequestLimiter.
+ */
public class TestRequestLimiter {
class MockRequest {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
index 26cf979..50fda95 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
@@ -17,24 +17,35 @@
*/
package org.apache.distributedlog.lock;
+import static org.apache.distributedlog.lock.ZKSessionLock.asyncParseClientID;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.distributedlog.util.FailpointUtils;
-import org.apache.distributedlog.exceptions.LockingException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.distributedlog.TestDistributedLogBase;
-import org.apache.distributedlog.common.concurrent.FutureEventListener;
-import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.distributedlog.util.Utils;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.ZooKeeperClientBuilder;
import org.apache.distributedlog.ZooKeeperClientUtils;
+import org.apache.distributedlog.common.concurrent.FutureEventListener;
+import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.commons.lang3.tuple.Pair;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.ZooKeeper;
-
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
@@ -43,31 +54,18 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.apache.distributedlog.lock.ZKSessionLock.asyncParseClientID;
/**
- * Distributed Lock Tests
+ * Distributed Lock Tests.
*/
public class TestDistributedLock extends TestDistributedLogBase {
- static final Logger logger = LoggerFactory.getLogger(TestDistributedLock.class);
+ private static final Logger logger = LoggerFactory.getLogger(TestDistributedLock.class);
@Rule
public TestName runtime = new TestName();
- private final static int sessionTimeoutMs = 2000;
+ private static final int sessionTimeoutMs = 2000;
private ZooKeeperClient zkc;
private ZooKeeperClient zkc0; // used for checking
@@ -313,7 +311,8 @@
@Test(timeout = 60000)
public void testCheckWriteLockFailureWhenLockIsAcquiredByOthers() throws Exception {
- String lockPath = "/test-check-write-lock-failure-when-lock-is-acquired-by-others-" + System.currentTimeMillis();
+ String lockPath = "/test-check-write-lock-failure-when-lock-is-acquired-by-others-"
+ + System.currentTimeMillis();
String clientId = "test-check-write-lock-failure";
createLockPath(zkc.get(), lockPath);
@@ -724,7 +723,7 @@
// acquired).
for (int i = 0; i < count; i++) {
latches[i].await();
- assertLatchesSet(latches, i+1);
+ assertLatchesSet(latches, i + 1);
Utils.ioResult(results.get(i));
Utils.ioResult(lockArray[i].asyncClose());
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
index 5189104..ea13cce 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
@@ -17,20 +17,32 @@
*/
package org.apache.distributedlog.lock;
+import static org.apache.distributedlog.lock.ZKSessionLock.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.distributedlog.DLMTestUtil;
-import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.ZooKeeperClientBuilder;
import org.apache.distributedlog.ZooKeeperClientUtils;
import org.apache.distributedlog.ZooKeeperClusterTestCase;
+import org.apache.distributedlog.exceptions.LockingException;
import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
import org.apache.distributedlog.lock.ZKSessionLock.State;
import org.apache.distributedlog.util.FailpointUtils;
import org.apache.distributedlog.util.OrderedScheduler;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.util.SafeRunnable;
-import org.apache.commons.lang3.tuple.Pair;
import org.apache.distributedlog.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
@@ -44,31 +56,21 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import static org.apache.distributedlog.lock.ZKSessionLock.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+
+
/**
- * Distributed Lock Tests
+ * Distributed Lock Tests.
*/
public class TestZKSessionLock extends ZooKeeperClusterTestCase {
@Rule
public TestName testNames = new TestName();
- static final Logger logger = LoggerFactory.getLogger(TestZKSessionLock.class);
+ private static final Logger logger = LoggerFactory.getLogger(TestZKSessionLock.class);
- private final static int sessionTimeoutMs = 2000;
+ private static final int sessionTimeoutMs = 2000;
private ZooKeeperClient zkc;
private ZooKeeperClient zkc0; // used for checking
@@ -121,8 +123,8 @@
ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
}
- private static String createLockNodeWithBadNodeName(ZooKeeper zk, String lockPath, String clientId, String badNodeName)
- throws Exception {
+ private static String createLockNodeWithBadNodeName(ZooKeeper zk, String lockPath,
+ String clientId, String badNodeName) throws Exception {
return zk.create(lockPath + "/" + badNodeName, serializeClientId(clientId),
ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
}
@@ -155,9 +157,12 @@
// Bad Lock Node Name
String node4 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member"));
String node5 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode"));
- String node6 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode"));
- String node7 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode_badnode"));
- String node8 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode_badnode_badnode"));
+ String node6 = getLockIdFromPath(
+ createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode"));
+ String node7 = getLockIdFromPath(
+ createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode_badnode"));
+ String node8 = getLockIdFromPath(
+ createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode_badnode_badnode"));
assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node4)));
assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node5)));
@@ -166,7 +171,8 @@
assertEquals(lockId, Utils.ioResult(asyncParseClientID(zk, lockPath, node8)));
// Malformed Node Name
- String node9 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_malformed_s12345678_999999"));
+ String node9 = getLockIdFromPath(
+ createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_malformed_s12345678_999999"));
assertEquals(Pair.of("malformed", 12345678L), Utils.ioResult(asyncParseClientID(zk, lockPath, node9)));
}
@@ -339,7 +345,7 @@
ZKSessionLock lock = new ZKSessionLock(
zkc, lockPath, clientId, lockStateExecutor,
- 1*1000 /* op timeout */, NullStatsLogger.INSTANCE,
+ 1 * 1000 /* op timeout */, NullStatsLogger.INSTANCE,
new DistributedLockContext());
lock.tryLock(0, TimeUnit.MILLISECONDS);
@@ -347,7 +353,7 @@
try {
FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_LockUnlockCleanup,
- new DelayFailpointAction(60*60*1000));
+ new DelayFailpointAction(60 * 60 * 1000));
lock.unlock();
assertEquals(State.CLOSING, lock.getLockState());
@@ -371,7 +377,7 @@
ZKSessionLock lock = new ZKSessionLock(
zkc, lockPath, clientId, lockStateExecutor,
- 1*1000 /* op timeout */, NullStatsLogger.INSTANCE,
+ 1 * 1000 /* op timeout */, NullStatsLogger.INSTANCE,
new DistributedLockContext());
try {
@@ -380,7 +386,6 @@
lock.tryLock(0, TimeUnit.MILLISECONDS);
} catch (LockClosedException ex) {
- ;
} finally {
FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition);
}
@@ -410,7 +415,7 @@
try {
FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_LockTryAcquire,
- new DelayFailpointAction(60*60*1000));
+ new DelayFailpointAction(60 * 60 * 1000));
lock.tryLock(0, TimeUnit.MILLISECONDS);
assertEquals(State.CLOSED, lock.getLockState());
@@ -433,8 +438,7 @@
}
/**
- * Test Basic Lock and Unlock
- *
+ * Test Basic Lock and Unlock.
* - lock should succeed if there is no lock held
* - lock should fail on a success lock
* - unlock should release the held lock
@@ -479,7 +483,6 @@
/**
* Test lock on non existed lock.
- *
* - lock should fail on a non existed lock.
*
* @throws Exception
@@ -580,8 +583,8 @@
@Test(timeout = 60000)
public void testLockWhenPreviousLockZnodeStillExists() throws Exception {
- String lockPath = "/test-lock-when-previous-lock-znode-still-exists-" +
- System.currentTimeMillis();
+ String lockPath = "/test-lock-when-previous-lock-znode-still-exists-"
+ + System.currentTimeMillis();
String clientId = "client-id";
ZooKeeper zk = zkc.get();
@@ -871,7 +874,8 @@
}
private void testLockUseSameClientIdButDifferentSessions(boolean isUnlock) throws Exception {
- String lockPath = "/test-lock-use-same-client-id-but-different-sessions-" + isUnlock + System.currentTimeMillis();
+ String lockPath = "/test-lock-use-same-client-id-but-different-sessions-"
+ + isUnlock + System.currentTimeMillis();
String clientId = "test-lock-use-same-client-id-but-different-sessions";
createLockPath(zkc.get(), lockPath);
@@ -970,7 +974,7 @@
}
/**
- * Immediate lock and unlock first lock
+ * Immediate lock and unlock first lock.
* @throws Exception
*/
@Test(timeout = 60000)
@@ -979,7 +983,7 @@
}
/**
- * Immediate lock and expire first lock
+ * Immediate lock and expire first lock.
* @throws Exception
*/
@Test(timeout = 60000)
@@ -988,7 +992,7 @@
}
/**
- * Wait Lock and unlock lock0_0 and lock1
+ * Wait Lock and unlock lock0_0 and lock1.
* @throws Exception
*/
@Test(timeout = 60000)
@@ -997,7 +1001,7 @@
}
/**
- * Wait Lock and expire first & third lock
+ * Wait Lock and expire first & third lock.
* @throws Exception
*/
@Test(timeout = 60000)
@@ -1057,7 +1061,8 @@
children = getLockWaiters(zkc, lockPath);
assertEquals(2, children.size());
assertEquals(State.CLAIMED, lock0_0.getLockState());
- assertEquals(lock0_0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+ assertEquals(lock0_0.getLockId(),
+ Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
assertEquals(State.WAITING, lock1.getLockState());
assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
} else {
@@ -1086,11 +1091,13 @@
assertEquals(3, children.size());
assertEquals(State.CLAIMED, lock0_0.getLockState());
- assertEquals(lock0_0.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+ assertEquals(lock0_0.getLockId(),
+ Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
awaitState(State.WAITING, lock1);
assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
awaitState(State.WAITING, lock0_1);
- assertEquals(lock0_1.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(2))));
+ assertEquals(lock0_1.getLockId(),
+ Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(2))));
}
if (isUnlock) {
@@ -1127,14 +1134,17 @@
children = getLockWaiters(zkc, lockPath);
assertEquals(1, children.size());
assertEquals(State.CLAIMED, lock1.getLockState());
- assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+ assertEquals(lock1.getLockId(),
+ Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
} else {
children = getLockWaiters(zkc, lockPath);
assertEquals(2, children.size());
assertEquals(State.CLAIMED, lock1.getLockState());
- assertEquals(lock1.getLockId(), Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+ assertEquals(lock1.getLockId(),
+ Utils.ioResult(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
assertEquals(State.WAITING, lock0_1.getLockState());
- assertEquals(lock0_1.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
+ assertEquals(lock0_1.getLockId(),
+ Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
}
}
@@ -1147,7 +1157,8 @@
children = getLockWaiters(zkc, lockPath);
assertEquals(1, children.size());
assertEquals(State.CLAIMED, lock0_1.getLockState());
- assertEquals(lock0_1.getLockId(), Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+ assertEquals(lock0_1.getLockId(),
+ Utils.ioResult(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
}
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java b/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
index 4370687..d79250c 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
@@ -17,20 +17,21 @@
*/
package org.apache.distributedlog.logsegment;
+import static org.junit.Assert.*;
+
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
-import org.apache.distributedlog.DLMTestUtil;
-import org.apache.distributedlog.LogSegmentMetadata;
-import org.apache.distributedlog.exceptions.UnexpectedException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.junit.Test;
-
import java.util.List;
import java.util.Map;
import java.util.Set;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.junit.Test;
-import static org.junit.Assert.*;
+
/**
* Test Case for Per Stream Log Segment Cache.
@@ -160,7 +161,7 @@
cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(1L), segment1);
LogSegmentMetadata segment3 =
DLMTestUtil.completedLogSegment("/segment-3", 3L, 3L, 300L, 100, 3L, 99L, 0L)
- .mutator().setVersion(LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO).build();
+ .mutator().setVersion(LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO).build();
cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(3L), segment3);
List<LogSegmentMetadata> expectedList = Lists.asList(segment1, new LogSegmentMetadata[] { segment3 });
List<LogSegmentMetadata> resultList = cache.getLogSegments(LogSegmentMetadata.COMPARATOR);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java b/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java
index 6687b7b..4cfedbb 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.java
@@ -17,10 +17,10 @@
*/
package org.apache.distributedlog.logsegment;
+import static org.junit.Assert.*;
import org.apache.distributedlog.common.util.Sizable;
import org.junit.Test;
-import static org.junit.Assert.*;
/**
* Test Case for {@link RollingPolicy}s.
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java
index 489c66b..0209752 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java
@@ -17,6 +17,11 @@
*/
package org.apache.distributedlog.metadata;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import java.io.IOException;
+import java.net.URI;
import org.apache.distributedlog.LocalDLMEmulator;
import org.apache.distributedlog.ZooKeeperClusterTestCase;
import org.apache.distributedlog.impl.metadata.BKDLConfig;
@@ -26,12 +31,12 @@
import org.junit.Before;
import org.junit.Test;
-import java.io.IOException;
-import java.net.URI;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+
+/**
+ * Test Case for {@link DLMetadata}s.
+ */
public class TestDLMetadata extends ZooKeeperClusterTestCase {
private static final BKDLConfig bkdlConfig =
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java
index 5216861..d7fbb00 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java
@@ -17,14 +17,19 @@
*/
package org.apache.distributedlog.metadata;
-import org.apache.distributedlog.DLMTestUtil;
-import org.junit.Test;
-
-import java.net.URI;
import static org.apache.distributedlog.metadata.LogMetadata.*;
import static org.junit.Assert.*;
+import java.net.URI;
+import org.apache.distributedlog.DLMTestUtil;
+import org.junit.Test;
+
+
+
+/**
+ * Test Case for {@link LogMetadata}s.
+ */
public class TestLogMetadata {
@Test(timeout = 60000)
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
index 2090828..48ecde3 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
@@ -17,6 +17,10 @@
*/
package org.apache.distributedlog.metadata;
+import static org.junit.Assert.*;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.DistributedLogConfiguration;
@@ -37,12 +41,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.Map;
-import static org.junit.Assert.*;
+/**
+ * Test update for {@link LogSegmentMetadataStore}s.
+ */
public class TestLogSegmentMetadataStoreUpdater extends ZooKeeperClusterTestCase {
static final Logger LOG = LoggerFactory.getLogger(TestLogSegmentMetadataStoreUpdater.class);
@@ -88,14 +91,16 @@
Map<Long, LogSegmentMetadata> completedLogSegments = new HashMap<Long, LogSegmentMetadata>();
// Create 5 completed log segments
for (int i = 1; i <= 5; i++) {
- LogSegmentMetadata segment = DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0);
- completedLogSegments.put(((long)i), segment);
+ LogSegmentMetadata segment =
+ DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0);
+ completedLogSegments.put(((long) i), segment);
LOG.info("Create completed segment {} : {}", segment.getZkPath(), segment);
segment.write(zkc);
}
// Create a smaller inprogress log segment
long inprogressSeqNo = 3;
- LogSegmentMetadata segment = DLMTestUtil.inprogressLogSegment(ledgerPath, inprogressSeqNo, 5 * 100, inprogressSeqNo);
+ LogSegmentMetadata segment =
+ DLMTestUtil.inprogressLogSegment(ledgerPath, inprogressSeqNo, 5 * 100, inprogressSeqNo);
LOG.info("Create inprogress segment {} : {}", segment.getZkPath(), segment);
segment.write(zkc);
@@ -119,9 +124,9 @@
// check first 5 log segments
for (int i = 1; i <= 5; i++) {
- LogSegmentMetadata s = segmentList.get((long)i);
+ LogSegmentMetadata s = segmentList.get((long) i);
assertNotNull(s);
- assertEquals(completedLogSegments.get((long)i), s);
+ assertEquals(completedLogSegments.get((long) i), s);
}
// get log segment 6
@@ -138,7 +143,8 @@
String ledgerPath = "/testUpdateLastDLSN";
zkc.get().create(ledgerPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
// Create 1 completed log segment
- LogSegmentMetadata completedLogSegment = DLMTestUtil.completedLogSegment(ledgerPath, 1L, 0L, 99L, 100, 1L, 99L, 0L);
+ LogSegmentMetadata completedLogSegment =
+ DLMTestUtil.completedLogSegment(ledgerPath, 1L, 0L, 99L, 100, 1L, 99L, 0L);
completedLogSegment.write(zkc);
// Create 1 inprogress log segment
LogSegmentMetadata inprogressLogSegment = DLMTestUtil.inprogressLogSegment(ledgerPath, 2L, 100L, 2L);
@@ -214,7 +220,8 @@
assertEquals(completedLogSegment.getCompletionTime(), readCompletedLogSegment.getCompletionTime());
assertEquals(completedLogSegment.getFirstTxId(), readCompletedLogSegment.getFirstTxId());
assertEquals(completedLogSegment.getLogSegmentId(), readCompletedLogSegment.getLogSegmentId());
- assertEquals(completedLogSegment.getLogSegmentSequenceNumber(), readCompletedLogSegment.getLogSegmentSequenceNumber());
+ assertEquals(completedLogSegment.getLogSegmentSequenceNumber(),
+ readCompletedLogSegment.getLogSegmentSequenceNumber());
assertEquals(completedLogSegment.getRegionId(), readCompletedLogSegment.getRegionId());
assertEquals(completedLogSegment.getZkPath(), readCompletedLogSegment.getZkPath());
assertEquals(completedLogSegment.getZNodeName(), readCompletedLogSegment.getZNodeName());
@@ -231,8 +238,9 @@
Map<Long, LogSegmentMetadata> completedLogSegments = new HashMap<Long, LogSegmentMetadata>();
// Create 5 completed log segments
for (int i = 1; i <= 5; i++) {
- LogSegmentMetadata segment = DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0);
- completedLogSegments.put(((long)i), segment);
+ LogSegmentMetadata segment =
+ DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0);
+ completedLogSegments.put(((long) i), segment);
LOG.info("Create completed segment {} : {}", segment.getZkPath(), segment);
segment.write(zkc);
}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java b/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java
index 89b4852..fe99362 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestNamespaceBuilder.java
@@ -17,6 +17,9 @@
*/
package org.apache.distributedlog.namespace;
+import static org.apache.distributedlog.LocalDLMEmulator.DLOG_NAMESPACE;
+import static org.junit.Assert.assertTrue;
+import java.net.URI;
import org.apache.distributedlog.BKDistributedLogNamespace;
import org.apache.distributedlog.DistributedLogConfiguration;
import org.apache.distributedlog.TestDistributedLogBase;
@@ -24,13 +27,11 @@
import org.apache.distributedlog.api.namespace.NamespaceBuilder;
import org.junit.Test;
-import java.net.URI;
-import static org.apache.distributedlog.LocalDLMEmulator.DLOG_NAMESPACE;
-import static org.junit.Assert.assertTrue;
+
/**
- * Test Namespace Builder
+ * Test Namespace Builder.
*/
public class TestNamespaceBuilder extends TestDistributedLogBase {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java b/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java
index 3a3a850..b32f55e 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java
@@ -17,13 +17,16 @@
*/
package org.apache.distributedlog.net;
-import org.junit.Test;
-
+import static org.junit.Assert.*;
import java.util.ArrayList;
import java.util.List;
+import org.junit.Test;
-import static org.junit.Assert.*;
+
+/**
+ * Test update for {@link DNSResolver}s.
+ */
public class TestDNSResolver {
private static final String host1 = "r1-w1rack1-1111-2222.distributedlog.io";
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java
index 65ab9d9..fec785a 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java
@@ -17,12 +17,11 @@
*/
package org.apache.distributedlog.net;
+import static org.junit.Assert.*;
+import java.util.List;
import org.apache.bookkeeper.net.DNSToSwitchMapping;
import org.junit.Test;
-import java.util.List;
-
-import static org.junit.Assert.*;
/**
* Test Cases of {@link NetUtils}.
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java b/distributedlog-core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java
index 53147bd..d3ed329 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java
@@ -17,11 +17,11 @@
*/
package org.apache.distributedlog.selector;
+import static org.junit.Assert.*;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.DLSN;
import org.junit.Test;
-import static org.junit.Assert.*;
/**
* Test Case for {@link LogRecordSelector}s.
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java b/distributedlog-core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
index 71bf68d..f982da6 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
@@ -17,36 +17,51 @@
*/
package org.apache.distributedlog.tools;
+import static org.junit.Assert.*;
import java.net.URI;
+import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException;
+import org.apache.bookkeeper.util.ReflectionUtils;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.DLSN;
import org.apache.distributedlog.DistributedLogConfiguration;
-import org.apache.distributedlog.api.DistributedLogManager;
-import org.apache.distributedlog.TestDistributedLogBase;
import org.apache.distributedlog.LocalDLMEmulator;
import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.api.DistributedLogManager;
import org.apache.distributedlog.api.LogReader;
import org.apache.distributedlog.exceptions.ZKException;
-import org.apache.distributedlog.tools.DistributedLogTool.*;
-import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException;
-import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.distributedlog.tools.DistributedLogTool.CreateCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.DeleteAllocatorPoolCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.DeleteCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.DumpCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.InspectCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.ListCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.ReadEntriesCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.ReadLastConfirmedCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.ShowCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.TruncateCommand;
+import org.apache.distributedlog.tools.DistributedLogTool.TruncateStreamCommand;
+
+
import org.apache.zookeeper.KeeperException;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static org.junit.Assert.*;
+/**
+ * Test Case for {@link DistributedLogTool}s.
+ */
public class TestDistributedLogTool extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogTool.class);
- static final String defaultLedgerPath = LocalDLMEmulator.getBkLedgerPath();
- static final String defaultPath = "/test/namespace";
- static final String defaultHost = "127.0.0.1";
- static final String defaultPrivilegedZkAclId = "NathanielP";
+ private static final String defaultLedgerPath = LocalDLMEmulator.getBkLedgerPath();
+ private static final String defaultPath = "/test/namespace";
+ private static final String defaultHost = "127.0.0.1";
+ private static final String defaultPrivilegedZkAclId = "NathanielP";
static URI defaultUri = null;
static final String ADMIN_TOOL = org.apache.distributedlog.admin.DistributedLogAdmin.class.getName();
@@ -157,7 +172,6 @@
cmd.setStreamName("DefaultStream");
assertEquals(0, cmd.runCmd());
}
-
@Test(timeout = 60000)
public void testToolTruncate() throws Exception {
DistributedLogManager dlm = DLMTestUtil.createNewDLM("TruncateStream", conf, defaultUri);
@@ -212,7 +226,7 @@
DistributedLogManager dlm = DLMTestUtil.createNewDLM("testToolTruncateStream", confLocal, defaultUri);
DLMTestUtil.generateCompletedLogSegments(dlm, confLocal, 3, 1000);
- DLSN dlsn = new DLSN(2,1,0);
+ DLSN dlsn = new DLSN(2, 1, 0);
TruncateStreamCommand cmd = new TruncateStreamCommand();
cmd.setDlsn(dlsn);
cmd.setUri(defaultUri);
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java
index df7270e..f323966 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java
@@ -17,12 +17,15 @@
*/
package org.apache.distributedlog.util;
+import static org.junit.Assert.*;
import org.apache.commons.configuration.CompositeConfiguration;
import org.apache.commons.configuration.Configuration;
import org.junit.Test;
-import static org.junit.Assert.*;
+/**
+ * Test Case for {@link Configuration}s.
+ */
public class TestConfUtils {
@Test(timeout = 60000)
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java
index 92bb6f9..c712d67 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java
@@ -17,7 +17,11 @@
*/
package org.apache.distributedlog.util;
+import static com.google.common.base.Charsets.UTF_8;
+import static org.apache.distributedlog.util.DLUtils.validateAndNormalizeName;
+import static org.junit.Assert.*;
import com.google.common.collect.Lists;
+import java.util.List;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.LogSegmentMetadata;
import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
@@ -25,14 +29,11 @@
import org.apache.distributedlog.exceptions.UnexpectedException;
import org.junit.Test;
-import java.util.List;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.apache.distributedlog.util.DLUtils.validateAndNormalizeName;
-import static org.junit.Assert.*;
+
/**
- * Test Case for {@link DLUtils}
+ * Test Case for {@link DLUtils}.
*/
public class TestDLUtils {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
index 807ce02..ea6876f 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
@@ -17,18 +17,22 @@
*/
package org.apache.distributedlog.util;
-import org.apache.distributedlog.common.util.PermitManager;
-import org.apache.distributedlog.zk.LimitedPermitManager;
-import org.junit.Test;
-
+import static org.junit.Assert.*;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
+import org.apache.distributedlog.common.util.PermitManager;
+import org.apache.distributedlog.zk.LimitedPermitManager;
+import org.junit.Test;
-import static org.junit.Assert.*;
+
+
+/**
+ * Test Case for {@link PermitManager}.
+ */
public class TestPermitManager {
@Test(timeout = 60000)
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java
index ccb82c5..59dfe86 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java
@@ -17,9 +17,13 @@
*/
package org.apache.distributedlog.util;
-import org.junit.Test;
import static org.junit.Assert.*;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link TimeSequencer}.
+ */
public class TestTimeSequencer {
@Test(timeout = 60000)
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestUtils.java
index acd441c..752bc35 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestUtils.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestUtils.java
@@ -17,13 +17,16 @@
*/
package org.apache.distributedlog.util;
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
import com.google.common.base.Optional;
+import java.util.concurrent.CountDownLatch;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Versioned;
import org.apache.distributedlog.DLMTestUtil;
import org.apache.distributedlog.TestZooKeeperClientBuilder;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.ZooKeeperClusterTestCase;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Versioned;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs;
@@ -31,17 +34,15 @@
import org.junit.Before;
import org.junit.Test;
-import java.util.concurrent.CountDownLatch;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
+
/**
- * Test Utils
+ * Test Utils.
*/
public class TestUtils extends ZooKeeperClusterTestCase {
- private final static int sessionTimeoutMs = 30000;
+ private static final int sessionTimeoutMs = 30000;
private ZooKeeperClient zkc;
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java
index bb1fbb6..03d5279 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java
@@ -17,6 +17,10 @@
*/
package org.apache.distributedlog.zk;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+import java.util.concurrent.CountDownLatch;
+import javax.annotation.Nullable;
import org.apache.distributedlog.ZooKeeperClient;
import org.apache.distributedlog.exceptions.DLIllegalStateException;
import org.apache.zookeeper.KeeperException;
@@ -24,14 +28,11 @@
import org.apache.zookeeper.OpResult;
import org.junit.Test;
-import java.util.concurrent.CountDownLatch;
-import javax.annotation.Nullable;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+
/**
- * Test Case for zookeeper transaction
+ * Test Case for zookeeper transaction.
*/
public class TestZKTransaction {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java
index 064459e..d035630 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java
@@ -17,21 +17,22 @@
*/
package org.apache.distributedlog.zk;
-import org.apache.distributedlog.util.Transaction;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicReference;
import org.apache.bookkeeper.versioning.Version;
+import org.apache.distributedlog.util.Transaction;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.OpResult;
import org.junit.Test;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicReference;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+
/**
- * Test Case for versioned set operation
+ * Test Case for versioned set operation.
*/
public class TestZKVersionedSetOp {
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java
index ad62e2f..75f193b 100644
--- a/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java
@@ -17,17 +17,21 @@
*/
package org.apache.distributedlog.zk;
+import static org.junit.Assert.*;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.junit.Test;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import static org.junit.Assert.*;
+
+/**
+ * Test Case for {@link ZKWatcherManager}.
+ */
public class TestZKWatcherManager {
@Test(timeout = 60000)
diff --git a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java
index a1642f9..f5b166b 100644
--- a/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java
+++ b/distributedlog-proxy-server/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java
@@ -184,4 +184,4 @@
final DistributedLogServerApp launcher = new DistributedLogServerApp(args);
launcher.run();
}
-}
+}
\ No newline at end of file