DL-163: clean up direct zookeeper and bookkeeper usage and use metadata/data store abstraction
- introduce NamespaceDriver class to manage metadata/data store abstractions
- clean up direct zookeeper and bookkeeper usage and use metadata/data store abstraction
- separate MetadataAccessor from DistributedLogManager
- remove deprecated classes
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java
index 0f4d3d4..072c3ef 100644
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java
+++ b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java
@@ -25,7 +25,7 @@
import com.twitter.distributedlog.LogSegmentMetadata;
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
import java.io.IOException;
import java.util.List;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
index b9d0365..cebbc33 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
@@ -594,8 +594,6 @@
}
lastProcessTime.reset().start();
- lastProcessTime.reset().start();
-
// If the oldest pending promise is interrupted then we must mark
// the reader in error and abort all pending reads since we dont
// know the last consumed read
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
index a6b5fd2..9432e8a 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
@@ -24,7 +24,6 @@
import com.twitter.distributedlog.exceptions.WriteCancelledException;
import com.twitter.distributedlog.exceptions.WriteException;
import com.twitter.distributedlog.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.stats.OpStatsListener;
import com.twitter.distributedlog.util.FailpointUtils;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.util.Future;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
index d20cc6a..a3959b0 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
@@ -20,12 +20,6 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
-import com.google.common.base.Ticker;
-import com.twitter.distributedlog.bk.DynamicQuorumConfigProvider;
-import com.twitter.distributedlog.bk.LedgerAllocator;
-import com.twitter.distributedlog.bk.LedgerAllocatorDelegator;
-import com.twitter.distributedlog.bk.QuorumConfigProvider;
-import com.twitter.distributedlog.bk.SimpleLedgerAllocator;
import com.twitter.distributedlog.callback.LogSegmentListener;
import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
import com.twitter.distributedlog.exceptions.AlreadyClosedException;
@@ -34,27 +28,22 @@
import com.twitter.distributedlog.exceptions.UnexpectedException;
import com.twitter.distributedlog.function.CloseAsyncCloseableFunction;
import com.twitter.distributedlog.function.GetVersionedValueFunction;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.injector.AsyncRandomFailureInjector;
import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
import com.twitter.distributedlog.metadata.LogMetadataForReader;
import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.impl.metadata.ZKLogStreamMetadataStore;
import com.twitter.distributedlog.io.AsyncCloseable;
import com.twitter.distributedlog.lock.DistributedLock;
import com.twitter.distributedlog.lock.NopDistributedLock;
import com.twitter.distributedlog.lock.ZKDistributedLock;
import com.twitter.distributedlog.logsegment.LogSegmentFilter;
import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.metadata.BKDLConfig;
import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
import com.twitter.distributedlog.stats.BroadCastStatsLogger;
-import com.twitter.distributedlog.subscription.SubscriptionStateStore;
import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.distributedlog.subscription.ZKSubscriptionStateStore;
-import com.twitter.distributedlog.subscription.ZKSubscriptionsStore;
-import com.twitter.distributedlog.util.ConfUtils;
+import com.twitter.distributedlog.util.Allocator;
import com.twitter.distributedlog.util.DLUtils;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.MonitoredFuturePool;
@@ -65,24 +54,20 @@
import com.twitter.distributedlog.util.Utils;
import com.twitter.util.ExceptionalFunction;
import com.twitter.util.ExceptionalFunction0;
-import com.twitter.util.ExecutorServiceFuturePool;
import com.twitter.util.Function;
import com.twitter.util.Future;
-import com.twitter.util.FuturePool;
import com.twitter.util.FutureEventListener;
import com.twitter.util.Promise;
import org.apache.bookkeeper.feature.FeatureProvider;
import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
-import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
-import org.jboss.netty.util.HashedWheelTimer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.runtime.AbstractFunction0;
import scala.runtime.AbstractFunction1;
import scala.runtime.BoxedUnit;
+import java.io.Closeable;
import java.io.IOException;
import java.net.URI;
import java.util.HashSet;
@@ -91,6 +76,9 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
+import static com.twitter.distributedlog.namespace.NamespaceDriver.Role.READER;
+import static com.twitter.distributedlog.namespace.NamespaceDriver.Role.WRITER;
+
/**
* <h3>Metrics</h3>
* <ul>
@@ -112,7 +100,7 @@
* for details.
* </ul>
*/
-class BKDistributedLogManager extends ZKMetadataAccessor implements DistributedLogManager {
+class BKDistributedLogManager implements DistributedLogManager {
static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogManager.class);
static final Function<LogRecordWithDLSN, Long> RECORD_2_TXID_FUNCTION =
@@ -131,44 +119,28 @@
}
};
+ private final URI uri;
+ private final String name;
private final String clientId;
private final int regionId;
private final String streamIdentifier;
private final DistributedLogConfiguration conf;
private final DynamicDistributedLogConfiguration dynConf;
+ private final NamespaceDriver driver;
private Promise<Void> closePromise;
private final OrderedScheduler scheduler;
- private final OrderedScheduler readAheadScheduler;
- private boolean ownExecutor;
private final FeatureProvider featureProvider;
+ private final AsyncFailureInjector failureInjector;
private final StatsLogger statsLogger;
private final StatsLogger perLogStatsLogger;
final AlertStatsLogger alertStatsLogger;
- // log stream metadata stores
- private final LogStreamMetadataStore writerMetadataStore;
- private final LogStreamMetadataStore readerMetadataStore;
// log segment metadata cache
private final LogSegmentMetadataCache logSegmentMetadataCache;
- // bookkeeper clients
- // NOTE: The actual bookkeeper client is initialized lazily when it is referenced by
- // {@link com.twitter.distributedlog.BookKeeperClient#get()}. So it is safe to
- // keep builders and their client wrappers here, as they will be used when
- // instantiating readers or writers.
- private final BookKeeperClientBuilder writerBKCBuilder;
- private final BookKeeperClient writerBKC;
- private final LogSegmentEntryStore writerEntryStore;
- private final boolean ownWriterBKC;
- private final BookKeeperClientBuilder readerBKCBuilder;
- private final BookKeeperClient readerBKC;
- private final LogSegmentEntryStore readerEntryStore;
- private final boolean ownReaderBKC;
-
//
// Writer Related Variables
//
- private final LedgerAllocator ledgerAllocator;
private final PermitLimiter writeLimiter;
//
@@ -176,92 +148,26 @@
///
// read handler for listener.
private BKLogReadHandler readHandlerForListener = null;
- private FuturePool readerFuturePool = null;
private final PendingReaders pendingReaders;
- // Failure Injector
- private final AsyncFailureInjector failureInjector;
-
- /**
- * Create a DLM for testing.
- *
- * @param name log name
- * @param conf distributedlog configuration
- * @param uri uri location for the log
- * @param writerZKCBuilder zookeeper builder for writers
- * @param readerZKCBuilder zookeeper builder for readers
- * @param zkcForWriterBKC zookeeper builder for bookkeeper shared by writers
- * @param zkcForReaderBKC zookeeper builder for bookkeeper shared by readers
- * @param writerBKCBuilder bookkeeper builder for writers
- * @param readerBKCBuilder bookkeeper builder for readers
- * @param featureProvider provider to offer features
- * @param writeLimiter write limiter
- * @param statsLogger stats logger to receive stats
- * @throws IOException
- */
- BKDistributedLogManager(String name,
- DistributedLogConfiguration conf,
- URI uri,
- ZooKeeperClientBuilder writerZKCBuilder,
- ZooKeeperClientBuilder readerZKCBuilder,
- ZooKeeperClient zkcForWriterBKC,
- ZooKeeperClient zkcForReaderBKC,
- BookKeeperClientBuilder writerBKCBuilder,
- BookKeeperClientBuilder readerBKCBuilder,
- FeatureProvider featureProvider,
- PermitLimiter writeLimiter,
- StatsLogger statsLogger) throws IOException {
- this(name,
- conf,
- ConfUtils.getConstDynConf(conf),
- uri,
- writerZKCBuilder,
- readerZKCBuilder,
- zkcForWriterBKC,
- zkcForReaderBKC,
- writerBKCBuilder,
- readerBKCBuilder,
- null,
- null,
- new LogSegmentMetadataCache(conf, Ticker.systemTicker()),
- OrderedScheduler.newBuilder().name("BKDL-" + name).corePoolSize(1).build(),
- null,
- null,
- null,
- DistributedLogConstants.UNKNOWN_CLIENT_ID,
- DistributedLogConstants.LOCAL_REGION_ID,
- null,
- writeLimiter,
- featureProvider,
- statsLogger,
- NullStatsLogger.INSTANCE);
- this.ownExecutor = true;
- }
+ // resource to close
+ private final Optional<AsyncCloseable> resourcesCloseable;
/**
* Create a {@link DistributedLogManager} with supplied resources.
*
* @param name log name
* @param conf distributedlog configuration
+ * @param dynConf dynamic distributedlog configuration
* @param uri uri location for the log
- * @param writerZKCBuilder zookeeper builder for writers
- * @param readerZKCBuilder zookeeper builder for readers
- * @param zkcForWriterBKC zookeeper builder for bookkeeper shared by writers
- * @param zkcForReaderBKC zookeeper builder for bookkeeper shared by readers
- * @param writerBKCBuilder bookkeeper builder for writers
- * @param readerBKCBuilder bookkeeper builder for readers
- * @param writerMetadataStore writer metadata store
- * @param readerMetadataStore reader metadata store
+ * @param driver namespace driver
+ * @param logSegmentMetadataCache log segment metadata cache
* @param scheduler ordered scheduled used by readers and writers
- * @param readAheadScheduler readAhead scheduler used by readers
- * @param channelFactory client socket channel factory to build bookkeeper clients
- * @param requestTimer request timer to build bookkeeper clients
* @param clientId client id that used to initiate the locks
* @param regionId region id that would be encrypted as part of log segment metadata
* to indicate which region that the log segment will be created
- * @param ledgerAllocator ledger allocator to allocate ledgers
- * @param featureProvider provider to offer features
* @param writeLimiter write limiter
+ * @param featureProvider provider to offer features
* @param statsLogger stats logger to receive stats
* @param perLogStatsLogger stats logger to receive per log stats
* @throws IOException
@@ -270,148 +176,48 @@
DistributedLogConfiguration conf,
DynamicDistributedLogConfiguration dynConf,
URI uri,
- ZooKeeperClientBuilder writerZKCBuilder,
- ZooKeeperClientBuilder readerZKCBuilder,
- ZooKeeperClient zkcForWriterBKC,
- ZooKeeperClient zkcForReaderBKC,
- BookKeeperClientBuilder writerBKCBuilder,
- BookKeeperClientBuilder readerBKCBuilder,
- LogStreamMetadataStore writerMetadataStore,
- LogStreamMetadataStore readerMetadataStore,
+ NamespaceDriver driver,
LogSegmentMetadataCache logSegmentMetadataCache,
OrderedScheduler scheduler,
- OrderedScheduler readAheadScheduler,
- ClientSocketChannelFactory channelFactory,
- HashedWheelTimer requestTimer,
String clientId,
Integer regionId,
- LedgerAllocator ledgerAllocator,
PermitLimiter writeLimiter,
FeatureProvider featureProvider,
+ AsyncFailureInjector failureInjector,
StatsLogger statsLogger,
- StatsLogger perLogStatsLogger) throws IOException {
- super(name, conf, uri, writerZKCBuilder, readerZKCBuilder, statsLogger);
+ StatsLogger perLogStatsLogger,
+ Optional<AsyncCloseable> resourcesCloseable) {
+ this.name = name;
this.conf = conf;
this.dynConf = dynConf;
+ this.uri = uri;
+ this.driver = driver;
+ this.logSegmentMetadataCache = logSegmentMetadataCache;
this.scheduler = scheduler;
- this.readAheadScheduler = null == readAheadScheduler ? scheduler : readAheadScheduler;
this.statsLogger = statsLogger;
this.perLogStatsLogger = BroadCastStatsLogger.masterslave(perLogStatsLogger, statsLogger);
- this.ownExecutor = false;
this.pendingReaders = new PendingReaders(scheduler);
this.regionId = regionId;
this.clientId = clientId;
this.streamIdentifier = conf.getUnpartitionedStreamName();
- this.ledgerAllocator = ledgerAllocator;
this.writeLimiter = writeLimiter;
-
- // Failure Injection
- this.failureInjector = AsyncRandomFailureInjector.newBuilder()
- .injectDelays(conf.getEIInjectReadAheadDelay(),
- conf.getEIInjectReadAheadDelayPercent(),
- conf.getEIInjectMaxReadAheadDelayMs())
- .injectErrors(false, 10)
- .injectStops(conf.getEIInjectReadAheadStall(), 10)
- .injectCorruption(conf.getEIInjectReadAheadBrokenEntries())
- .build();
-
- if (null == writerMetadataStore) {
- this.writerMetadataStore = new ZKLogStreamMetadataStore(
- clientId,
- conf,
- writerZKC,
- scheduler,
- statsLogger);
- } else {
- this.writerMetadataStore = writerMetadataStore;
- }
- if (null == readerMetadataStore) {
- this.readerMetadataStore = new ZKLogStreamMetadataStore(
- clientId,
- conf,
- readerZKC,
- scheduler,
- statsLogger);
- } else {
- this.readerMetadataStore = readerMetadataStore;
- }
- this.logSegmentMetadataCache = logSegmentMetadataCache;
-
- // create the bkc for writers
- if (null == writerBKCBuilder) {
- // resolve uri
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(writerZKC, uri);
- BKDLConfig.propagateConfiguration(bkdlConfig, conf);
- this.writerBKCBuilder = BookKeeperClientBuilder.newBuilder()
- .dlConfig(conf)
- .name(String.format("bk:%s:dlm_writer_shared", name))
- .ledgersPath(bkdlConfig.getBkLedgersPath())
- .channelFactory(channelFactory)
- .requestTimer(requestTimer)
- .statsLogger(statsLogger);
- if (null == zkcForWriterBKC) {
- this.writerBKCBuilder.zkServers(bkdlConfig.getBkZkServersForWriter());
- } else {
- this.writerBKCBuilder.zkc(zkcForWriterBKC);
- }
- this.ownWriterBKC = true;
- } else {
- this.writerBKCBuilder = writerBKCBuilder;
- this.ownWriterBKC = false;
- }
- this.writerBKC = this.writerBKCBuilder.build();
- this.writerEntryStore = new BKLogSegmentEntryStore(
- conf,
- writerBKC,
- scheduler,
- statsLogger,
- failureInjector);
-
- // create the bkc for readers
- if (null == readerBKCBuilder) {
- // resolve uri
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(writerZKC, uri);
- BKDLConfig.propagateConfiguration(bkdlConfig, conf);
- if (bkdlConfig.getBkZkServersForWriter().equals(bkdlConfig.getBkZkServersForReader())) {
- this.readerBKCBuilder = this.writerBKCBuilder;
- this.ownReaderBKC = false;
- } else {
- this.readerBKCBuilder = BookKeeperClientBuilder.newBuilder()
- .dlConfig(conf)
- .name(String.format("bk:%s:dlm_reader_shared", name))
- .ledgersPath(bkdlConfig.getBkLedgersPath())
- .channelFactory(channelFactory)
- .requestTimer(requestTimer)
- .statsLogger(statsLogger);
- if (null == zkcForReaderBKC) {
- this.readerBKCBuilder.zkServers(bkdlConfig.getBkZkServersForReader());
- } else {
- this.readerBKCBuilder.zkc(zkcForReaderBKC);
- }
- this.ownReaderBKC = true;
- }
- } else {
- this.readerBKCBuilder = readerBKCBuilder;
- this.ownReaderBKC = false;
- }
- this.readerBKC = this.readerBKCBuilder.build();
- this.readerEntryStore = new BKLogSegmentEntryStore(
- conf,
- readerBKC,
- scheduler,
- statsLogger,
- failureInjector);
-
// Feature Provider
this.featureProvider = featureProvider;
-
+ // Failure Injector
+ this.failureInjector = failureInjector;
// Stats
this.alertStatsLogger = new AlertStatsLogger(this.perLogStatsLogger, "dl_alert");
+ this.resourcesCloseable = resourcesCloseable;
}
- @VisibleForTesting
- LogStreamMetadataStore getWriterMetadataStore() {
- return writerMetadataStore;
+ @Override
+ public String getStreamName() {
+ return name;
+ }
+
+ @Override
+ public NamespaceDriver getNamespaceDriver() {
+ return driver;
}
URI getUri() {
@@ -426,23 +232,22 @@
return scheduler;
}
+ AsyncFailureInjector getFailureInjector() {
+ return failureInjector;
+ }
+
+ //
+ // Test Methods
+ //
+
@VisibleForTesting
- BookKeeperClient getWriterBKC() {
- return this.writerBKC;
+ LogStreamMetadataStore getWriterMetadataStore() {
+ return driver.getLogStreamMetadataStore(WRITER);
}
@VisibleForTesting
- BookKeeperClient getReaderBKC() {
- return this.readerBKC;
- }
-
LogSegmentEntryStore getReaderEntryStore() {
- return this.readerEntryStore;
- }
-
- @VisibleForTesting
- FuturePool getReaderFuturePool() {
- return this.readerFuturePool;
+ return driver.getLogSegmentEntryStore(READER);
}
@VisibleForTesting
@@ -450,10 +255,6 @@
return this.featureProvider;
}
- AsyncFailureInjector getFailureInjector() {
- return this.failureInjector;
- }
-
private synchronized BKLogReadHandler getReadHandlerAndRegisterListener(
boolean create, LogSegmentListener listener) {
if (null == readHandlerForListener && create) {
@@ -502,12 +303,6 @@
throw new AlreadyClosedException("Executing " + operation + " on already closed DistributedLogManager");
}
}
- if (null != writerBKC) {
- writerBKC.checkClosedOrInError();
- }
- if (null != readerBKC) {
- readerBKC.checkClosedOrInError();
- }
}
// Create Read Handler
@@ -538,9 +333,9 @@
subscriberId,
conf,
dynConf,
- readerMetadataStore,
+ driver.getLogStreamMetadataStore(READER),
logSegmentMetadataCache,
- readerEntryStore,
+ driver.getLogSegmentEntryStore(READER),
scheduler,
alertStatsLogger,
statsLogger,
@@ -552,23 +347,7 @@
// Create Ledger Allocator
- LedgerAllocator createLedgerAllocator(LogMetadataForWriter logMetadata) throws IOException {
- LedgerAllocator ledgerAllocatorDelegator;
- if (!dynConf.getEnableLedgerAllocatorPool()) {
- QuorumConfigProvider quorumConfigProvider =
- new DynamicQuorumConfigProvider(dynConf);
- LedgerAllocator allocator = new SimpleLedgerAllocator(
- logMetadata.getAllocationPath(),
- logMetadata.getAllocationData(),
- quorumConfigProvider,
- writerZKC,
- writerBKC);
- ledgerAllocatorDelegator = new LedgerAllocatorDelegator(allocator, true);
- } else {
- ledgerAllocatorDelegator = ledgerAllocator;
- }
- return ledgerAllocatorDelegator;
- }
+
// Create Write Handler
@@ -578,14 +357,12 @@
}
Future<BKLogWriteHandler> asyncCreateWriteHandler(final boolean lockHandler) {
- boolean ownAllocator = null == ledgerAllocator;
-
// Fetching Log Metadata (create if not exists)
- return writerMetadataStore.getLog(
+ return driver.getLogStreamMetadataStore(WRITER).getLog(
uri,
name,
- ownAllocator,
- conf.getCreateStreamIfNotExists() || ownAllocator
+ true,
+ conf.getCreateStreamIfNotExists()
).flatMap(new AbstractFunction1<LogMetadataForWriter, Future<BKLogWriteHandler>>() {
@Override
public Future<BKLogWriteHandler> apply(LogMetadataForWriter logMetadata) {
@@ -602,16 +379,17 @@
// Build the locks
DistributedLock lock;
if (conf.isWriteLockEnabled()) {
- lock = writerMetadataStore.createWriteLock(logMetadata);
+ lock = driver.getLogStreamMetadataStore(WRITER).createWriteLock(logMetadata);
} else {
lock = NopDistributedLock.INSTANCE;
}
- // Build the ledger allocator
- LedgerAllocator allocator;
+
+ Allocator<LogSegmentEntryWriter, Object> segmentAllocator;
try {
- allocator = createLedgerAllocator(logMetadata);
- } catch (IOException e) {
- FutureUtils.setException(createPromise, e);
+ segmentAllocator = driver.getLogSegmentEntryStore(WRITER)
+ .newLogSegmentAllocator(logMetadata, dynConf);
+ } catch (IOException ioe) {
+ FutureUtils.setException(createPromise, ioe);
return;
}
@@ -619,11 +397,11 @@
final BKLogWriteHandler writeHandler = new BKLogWriteHandler(
logMetadata,
conf,
- writerMetadataStore,
+ driver.getLogStreamMetadataStore(WRITER),
logSegmentMetadataCache,
- writerEntryStore,
+ driver.getLogSegmentEntryStore(WRITER),
scheduler,
- allocator,
+ segmentAllocator,
statsLogger,
perLogStatsLogger,
alertStatsLogger,
@@ -657,12 +435,11 @@
}
PermitManager getLogSegmentRollingPermitManager() {
- return writerMetadataStore.getPermitManager();
+ return driver.getLogStreamMetadataStore(WRITER).getPermitManager();
}
<T> Future<T> processReaderOperation(final Function<BKLogReadHandler, Future<T>> func) {
- initializeFuturePool(false);
- return readerFuturePool.apply(new ExceptionalFunction0<BKLogReadHandler>() {
+ return scheduler.apply(new ExceptionalFunction0<BKLogReadHandler>() {
@Override
public BKLogReadHandler applyE() throws Throwable {
return getReadHandlerAndRegisterListener(true, null);
@@ -822,7 +599,7 @@
fromTxnId,
segmentIdx,
segments,
- readerEntryStore
+ driver.getLogSegmentEntryStore(READER)
);
}
@@ -1010,8 +787,9 @@
LOG.info("Reader {} @ {} reading last commit position from subscription store after acquired lock.",
subscriberId.get(), name);
// we acquired lock
- final SubscriptionStateStore stateStore = getSubscriptionStateStore(subscriberId.get());
- return stateStore.getLastCommitPosition().map(new ExceptionalFunction<DLSN, AsyncLogReader>() {
+ final SubscriptionsStore subscriptionsStore = driver.getSubscriptionsStore(getStreamName());
+ return subscriptionsStore.getLastCommitPosition(subscriberId.get())
+ .map(new ExceptionalFunction<DLSN, AsyncLogReader>() {
@Override
public AsyncLogReader applyE(DLSN lastCommitPosition) throws UnexpectedException {
LOG.info("Reader {} @ {} positioned to last commit position {}.",
@@ -1226,7 +1004,8 @@
*/
@Override
public void delete() throws IOException {
- FutureUtils.result(writerMetadataStore.deleteLog(uri, getStreamName()));
+ FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+ .deleteLog(uri, getStreamName()));
}
/**
@@ -1297,43 +1076,10 @@
readHandlerToClose = readHandlerForListener;
}
- // NOTE: the resources {scheduler, writerBKC, readerBKC} are mostly from namespace instance.
- // so they are not blocking call except tests.
- AsyncCloseable resourcesCloseable = new AsyncCloseable() {
- @Override
- public Future<Void> asyncClose() {
- int schedTimeout = conf.getSchedulerShutdownTimeoutMs();
-
- // Clean up executor state.
- if (ownExecutor) {
- SchedulerUtils.shutdownScheduler(scheduler, schedTimeout, TimeUnit.MILLISECONDS);
- LOG.info("Stopped BKDL executor service for {}.", name);
-
- if (scheduler != readAheadScheduler) {
- SchedulerUtils.shutdownScheduler(readAheadScheduler, schedTimeout, TimeUnit.MILLISECONDS);
- LOG.info("Stopped BKDL ReadAhead Executor Service for {}.", name);
- }
- }
- if (ownWriterBKC) {
- writerBKC.close();
- }
- if (ownReaderBKC) {
- readerBKC.close();
- }
- return Future.Void();
- }
- };
-
Future<Void> closeResult = Utils.closeSequence(null, true,
readHandlerToClose,
pendingReaders,
- resourcesCloseable,
- new AsyncCloseable() {
- @Override
- public Future<Void> asyncClose() {
- return BKDistributedLogManager.super.asyncClose();
- }
- });
+ resourcesCloseable.or(AsyncCloseable.NULL));
closeResult.proxyTo(closeFuture);
return closeFuture;
}
@@ -1343,70 +1089,18 @@
FutureUtils.result(asyncClose());
}
- private FuturePool buildFuturePool(ExecutorService executorService,
- StatsLogger statsLogger) {
- FuturePool futurePool = new ExecutorServiceFuturePool(executorService);
- return new MonitoredFuturePool(
- futurePool,
- statsLogger,
- conf.getEnableTaskExecutionStats(),
- conf.getTaskExecutionWarnTimeMicros());
- }
-
- private void initializeFuturePool(boolean ordered) {
- // ownExecutor is a single threaded thread pool
- if (null == readerFuturePool) {
- readerFuturePool = buildFuturePool(
- scheduler, statsLogger.scope("reader_future_pool"));
- }
- }
-
@Override
public String toString() {
- return String.format("DLM:%s:%s", getZKPath(), getStreamName());
+ return String.format("DLM:%s:%s", getUri(), getStreamName());
}
public void raiseAlert(String msg, Object... args) {
alertStatsLogger.raise(msg, args);
}
- /**
- * Get the subscription state storage provided by the distributed log manager
- *
- * @param subscriberId - Application specific Id associated with the subscriber
- * @return Subscription state store
- */
- @Override
- @Deprecated
- public SubscriptionStateStore getSubscriptionStateStore(String subscriberId) {
- return getSubscriptionStateStoreInternal(conf.getUnpartitionedStreamName(), subscriberId);
- }
-
- /**
- * Get the subscription state storage provided by the distributed log manager
- *
- * @param streamIdentifier - Identifier associated with the stream
- * @param subscriberId - Application specific Id associated with the subscriber
- * @return Subscription state store
- */
- private SubscriptionStateStore getSubscriptionStateStoreInternal(String streamIdentifier, String subscriberId) {
- return new ZKSubscriptionStateStore(writerZKC,
- LogMetadataForReader.getSubscriberPath(uri, name, streamIdentifier, subscriberId));
- }
-
@Override
public SubscriptionsStore getSubscriptionsStore() {
- return getSubscriptionsStoreInternal(conf.getUnpartitionedStreamName());
+ return driver.getSubscriptionsStore(getStreamName());
}
- /**
- * Get the subscription state storage provided by the distributed log manager
- *
- * @param streamIdentifier - Identifier associated with the stream
- * @return Subscriptions store
- */
- private SubscriptionsStore getSubscriptionsStoreInternal(String streamIdentifier) {
- return new ZKSubscriptionsStore(writerZKC,
- LogMetadataForReader.getSubscribersPath(uri, name, streamIdentifier));
- }
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
index 1a23228..a8b1f77 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
@@ -17,71 +17,39 @@
*/
package com.twitter.distributedlog;
-import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
import com.google.common.base.Ticker;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.DistributedLogManagerFactory.ClientSharingOption;
import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.acl.DefaultAccessControlManager;
-import com.twitter.distributedlog.acl.ZKAccessControlManager;
-import com.twitter.distributedlog.bk.LedgerAllocator;
-import com.twitter.distributedlog.bk.LedgerAllocatorUtils;
import com.twitter.distributedlog.callback.NamespaceListener;
import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
import com.twitter.distributedlog.exceptions.AlreadyClosedException;
import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.impl.ZKLogMetadataStore;
-import com.twitter.distributedlog.impl.federated.FederatedZKLogMetadataStore;
-import com.twitter.distributedlog.impl.metadata.ZKLogStreamMetadataStore;
+import com.twitter.distributedlog.injector.AsyncFailureInjector;
+import com.twitter.distributedlog.io.AsyncCloseable;
import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.LogMetadataStore;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.DLUtils;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor;
import com.twitter.distributedlog.util.OrderedScheduler;
import com.twitter.distributedlog.util.PermitLimiter;
import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.distributedlog.util.SimplePermitLimiter;
import com.twitter.distributedlog.util.Utils;
import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-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.ZooKeeper;
-import org.apache.zookeeper.common.PathUtils;
-import org.apache.zookeeper.data.Stat;
-import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
-import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
-import org.jboss.netty.util.HashedWheelTimer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
-import java.net.InetAddress;
import java.net.URI;
-import java.util.Collection;
-import java.util.HashMap;
import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
-import static com.twitter.distributedlog.impl.BKDLUtils.*;
+import static com.twitter.distributedlog.namespace.NamespaceDriver.Role.WRITER;
+import static com.twitter.distributedlog.util.DLUtils.validateName;
/**
* BKDistributedLogNamespace is the default implementation of {@link DistributedLogNamespace}. It uses
@@ -119,344 +87,57 @@
public class BKDistributedLogNamespace implements DistributedLogNamespace {
static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogNamespace.class);
- public static Builder newBuilder() {
- return new Builder();
- }
-
- public static class Builder {
- private DistributedLogConfiguration _conf = null;
- private URI _uri = null;
- private StatsLogger _statsLogger = NullStatsLogger.INSTANCE;
- private StatsLogger _perLogStatsLogger = NullStatsLogger.INSTANCE;
- private FeatureProvider _featureProvider = new SettableFeatureProvider("", 0);
- private String _clientId = DistributedLogConstants.UNKNOWN_CLIENT_ID;
- private int _regionId = DistributedLogConstants.LOCAL_REGION_ID;
-
- private Builder() {}
-
- public Builder conf(DistributedLogConfiguration conf) {
- this._conf = conf;
- return this;
- }
-
- public Builder uri(URI uri) {
- this._uri = uri;
- return this;
- }
-
- public Builder statsLogger(StatsLogger statsLogger) {
- this._statsLogger = statsLogger;
- return this;
- }
-
- public Builder perLogStatsLogger(StatsLogger perLogStatsLogger) {
- this._perLogStatsLogger = perLogStatsLogger;
- return this;
- }
-
- public Builder featureProvider(FeatureProvider featureProvider) {
- this._featureProvider = featureProvider;
- return this;
- }
-
- public Builder clientId(String clientId) {
- this._clientId = clientId;
- return this;
- }
-
- public Builder regionId(int regionId) {
- this._regionId = regionId;
- return this;
- }
-
- @SuppressWarnings("deprecation")
- public BKDistributedLogNamespace build()
- throws IOException, NullPointerException, IllegalArgumentException {
- Preconditions.checkNotNull(_conf, "No DistributedLog Configuration");
- Preconditions.checkNotNull(_uri, "No DistributedLog URI");
- Preconditions.checkNotNull(_featureProvider, "No Feature Provider");
- Preconditions.checkNotNull(_statsLogger, "No Stats Logger");
- Preconditions.checkNotNull(_featureProvider, "No Feature Provider");
- Preconditions.checkNotNull(_clientId, "No Client ID");
- // validate conf and uri
- validateConfAndURI(_conf, _uri);
-
- // Build the namespace zookeeper client
- ZooKeeperClientBuilder nsZkcBuilder = createDLZKClientBuilder(
- String.format("dlzk:%s:factory_writer_shared", _uri),
- _conf,
- DLUtils.getZKServersFromDLUri(_uri),
- _statsLogger.scope("dlzk_factory_writer_shared"));
- ZooKeeperClient nsZkc = nsZkcBuilder.build();
-
- // Resolve namespace binding
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(nsZkc, _uri);
-
- // Backward Compatible to enable per log stats by configuration settings
- StatsLogger perLogStatsLogger = _perLogStatsLogger;
- if (perLogStatsLogger == NullStatsLogger.INSTANCE &&
- _conf.getEnablePerStreamStat()) {
- perLogStatsLogger = _statsLogger.scope("stream");
- }
-
- return new BKDistributedLogNamespace(
- _conf,
- _uri,
- _featureProvider,
- _statsLogger,
- perLogStatsLogger,
- _clientId,
- _regionId,
- nsZkcBuilder,
- nsZkc,
- bkdlConfig);
- }
- }
-
- static interface ZooKeeperClientHandler<T> {
- T handle(ZooKeeperClient zkc) throws IOException;
- }
-
- /**
- * Run given <i>handler</i> by providing an available new zookeeper client
- *
- * @param handler
- * Handler to process with provided zookeeper client.
- * @param conf
- * Distributedlog Configuration.
- * @param namespace
- * Distributedlog Namespace.
- */
- private static <T> T withZooKeeperClient(ZooKeeperClientHandler<T> handler,
- DistributedLogConfiguration conf,
- URI namespace) throws IOException {
- ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
- .name(String.format("dlzk:%s:factory_static", namespace))
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .uri(namespace)
- .retryThreadCount(conf.getZKClientNumberRetryThreads())
- .requestRateLimit(conf.getZKRequestRateLimit())
- .zkAclId(conf.getZkAclId())
- .build();
- try {
- return handler.handle(zkc);
- } finally {
- zkc.close();
- }
- }
-
- private static String getHostIpLockClientId() {
- try {
- return InetAddress.getLocalHost().toString();
- } catch(Exception ex) {
- return DistributedLogConstants.UNKNOWN_CLIENT_ID;
- }
- }
-
private final String clientId;
private final int regionId;
private final DistributedLogConfiguration conf;
private final URI namespace;
- private final BKDLConfig bkdlConfig;
+ // namespace driver
+ private final NamespaceDriver driver;
+ // resources
private final OrderedScheduler scheduler;
- private final OrderedScheduler readAheadExecutor;
- private final ClientSocketChannelFactory channelFactory;
- private final HashedWheelTimer requestTimer;
- // zookeeper clients
- // NOTE: The actual zookeeper client is initialized lazily when it is referenced by
- // {@link com.twitter.distributedlog.ZooKeeperClient#get()}. So it is safe to
- // keep builders and their client wrappers here, as they will be used when
- // instantiating readers or writers.
- private final ZooKeeperClientBuilder sharedWriterZKCBuilderForDL;
- private final ZooKeeperClient sharedWriterZKCForDL;
- private final ZooKeeperClientBuilder sharedReaderZKCBuilderForDL;
- private final ZooKeeperClient sharedReaderZKCForDL;
- private ZooKeeperClientBuilder sharedWriterZKCBuilderForBK = null;
- private ZooKeeperClient sharedWriterZKCForBK = null;
- private ZooKeeperClientBuilder sharedReaderZKCBuilderForBK = null;
- private ZooKeeperClient sharedReaderZKCForBK = null;
- // NOTE: The actual bookkeeper client is initialized lazily when it is referenced by
- // {@link com.twitter.distributedlog.BookKeeperClient#get()}. So it is safe to
- // keep builders and their client wrappers here, as they will be used when
- // instantiating readers or writers.
- private final BookKeeperClientBuilder sharedWriterBKCBuilder;
- private final BookKeeperClient writerBKC;
- private final BookKeeperClientBuilder sharedReaderBKCBuilder;
- private final BookKeeperClient readerBKC;
- // ledger allocator
- private final LedgerAllocator allocator;
- // access control manager
- private AccessControlManager accessControlManager;
- // log metadata store
- private final LogMetadataStore metadataStore;
+ private final PermitLimiter writeLimiter;
+ private final AsyncFailureInjector failureInjector;
// log segment metadata store
private final LogSegmentMetadataCache logSegmentMetadataCache;
- private final LogStreamMetadataStore writerStreamMetadataStore;
- private final LogStreamMetadataStore readerStreamMetadataStore;
-
// feature provider
private final FeatureProvider featureProvider;
-
// Stats Loggers
private final StatsLogger statsLogger;
private final StatsLogger perLogStatsLogger;
- protected AtomicBoolean closed = new AtomicBoolean(false);
+ protected final AtomicBoolean closed = new AtomicBoolean(false);
- private final PermitLimiter writeLimiter;
-
- private BKDistributedLogNamespace(
+ public BKDistributedLogNamespace(
DistributedLogConfiguration conf,
URI uri,
+ NamespaceDriver driver,
+ OrderedScheduler scheduler,
FeatureProvider featureProvider,
+ PermitLimiter writeLimiter,
+ AsyncFailureInjector failureInjector,
StatsLogger statsLogger,
StatsLogger perLogStatsLogger,
String clientId,
- int regionId,
- ZooKeeperClientBuilder nsZkcBuilder,
- ZooKeeperClient nsZkc,
- BKDLConfig bkdlConfig)
- throws IOException, IllegalArgumentException {
+ int regionId) {
this.conf = conf;
this.namespace = uri;
+ this.driver = driver;
+ this.scheduler = scheduler;
this.featureProvider = featureProvider;
+ this.writeLimiter = writeLimiter;
+ this.failureInjector = failureInjector;
this.statsLogger = statsLogger;
this.perLogStatsLogger = perLogStatsLogger;
+ this.clientId = clientId;
this.regionId = regionId;
- this.bkdlConfig = bkdlConfig;
- if (clientId.equals(DistributedLogConstants.UNKNOWN_CLIENT_ID)) {
- this.clientId = getHostIpLockClientId();
- } else {
- this.clientId = clientId;
- }
- // Build resources
- StatsLogger schedulerStatsLogger = statsLogger.scope("factory").scope("thread_pool");
- this.scheduler = OrderedScheduler.newBuilder()
- .name("DLM-" + uri.getPath())
- .corePoolSize(conf.getNumWorkerThreads())
- .statsLogger(schedulerStatsLogger)
- .perExecutorStatsLogger(schedulerStatsLogger)
- .traceTaskExecution(conf.getEnableTaskExecutionStats())
- .traceTaskExecutionWarnTimeUs(conf.getTaskExecutionWarnTimeMicros())
- .build();
- if (conf.getNumReadAheadWorkerThreads() > 0) {
- this.readAheadExecutor = OrderedScheduler.newBuilder()
- .name("DLM-" + uri.getPath() + "-readahead-executor")
- .corePoolSize(conf.getNumReadAheadWorkerThreads())
- .statsLogger(statsLogger.scope("factory").scope("readahead_thread_pool"))
- .traceTaskExecution(conf.getTraceReadAheadDeliveryLatency())
- .traceTaskExecutionWarnTimeUs(conf.getTaskExecutionWarnTimeMicros())
- .build();
- LOG.info("Created dedicated readahead executor : threads = {}", conf.getNumReadAheadWorkerThreads());
- } else {
- this.readAheadExecutor = this.scheduler;
- LOG.info("Used shared executor for readahead.");
- }
-
- this.channelFactory = new NioClientSocketChannelFactory(
- Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-boss-%d").build()),
- Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-worker-%d").build()),
- conf.getBKClientNumberIOThreads());
- this.requestTimer = new HashedWheelTimer(
- new ThreadFactoryBuilder().setNameFormat("DLFactoryTimer-%d").build(),
- conf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS,
- conf.getTimeoutTimerNumTicks());
-
- // Build zookeeper client for writers
- this.sharedWriterZKCBuilderForDL = nsZkcBuilder;
- this.sharedWriterZKCForDL = nsZkc;
-
- // Build zookeeper client for readers
- if (bkdlConfig.getDlZkServersForWriter().equals(bkdlConfig.getDlZkServersForReader())) {
- this.sharedReaderZKCBuilderForDL = this.sharedWriterZKCBuilderForDL;
- } else {
- this.sharedReaderZKCBuilderForDL = createDLZKClientBuilder(
- String.format("dlzk:%s:factory_reader_shared", namespace),
- conf,
- bkdlConfig.getDlZkServersForReader(),
- statsLogger.scope("dlzk_factory_reader_shared"));
- }
- this.sharedReaderZKCForDL = this.sharedReaderZKCBuilderForDL.build();
-
- // Build bookkeeper client for writers
- this.sharedWriterBKCBuilder = createBKCBuilder(
- String.format("bk:%s:factory_writer_shared", namespace),
- conf,
- bkdlConfig.getBkZkServersForWriter(),
- bkdlConfig.getBkLedgersPath(),
- Optional.of(featureProvider.scope("bkc")));
- this.writerBKC = this.sharedWriterBKCBuilder.build();
-
- // Build bookkeeper client for readers
- if (bkdlConfig.getBkZkServersForWriter().equals(bkdlConfig.getBkZkServersForReader())) {
- this.sharedReaderBKCBuilder = this.sharedWriterBKCBuilder;
- } else {
- this.sharedReaderBKCBuilder = createBKCBuilder(
- String.format("bk:%s:factory_reader_shared", namespace),
- conf,
- bkdlConfig.getBkZkServersForReader(),
- bkdlConfig.getBkLedgersPath(),
- Optional.<FeatureProvider>absent());
- }
- this.readerBKC = this.sharedReaderBKCBuilder.build();
-
- if (conf.getGlobalOutstandingWriteLimit() < 0) {
- this.writeLimiter = PermitLimiter.NULL_PERMIT_LIMITER;
- } else {
- Feature disableWriteLimitFeature = featureProvider.getFeature(
- CoreFeatureKeys.DISABLE_WRITE_LIMIT.name().toLowerCase());
- this.writeLimiter = new SimplePermitLimiter(
- conf.getOutstandingWriteLimitDarkmode(),
- conf.getGlobalOutstandingWriteLimit(),
- statsLogger.scope("writeLimiter"),
- true /* singleton */,
- disableWriteLimitFeature);
- }
-
- // propagate bkdlConfig to configuration
- BKDLConfig.propagateConfiguration(bkdlConfig, conf);
-
- // Build the allocator
- if (conf.getEnableLedgerAllocatorPool()) {
- String allocatorPoolPath = validateAndGetFullLedgerAllocatorPoolPath(conf, uri);
- allocator = LedgerAllocatorUtils.createLedgerAllocatorPool(allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize(),
- conf, sharedWriterZKCForDL, writerBKC, scheduler);
- if (null != allocator) {
- allocator.start();
- }
- LOG.info("Created ledger allocator pool under {} with size {}.", allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize());
- } else {
- allocator = null;
- }
-
- // log metadata store
- if (bkdlConfig.isFederatedNamespace() || conf.isFederatedNamespaceEnabled()) {
- this.metadataStore = new FederatedZKLogMetadataStore(conf, namespace, sharedReaderZKCForDL, scheduler);
- } else {
- this.metadataStore = new ZKLogMetadataStore(conf, namespace, sharedReaderZKCForDL, scheduler);
- }
-
- // create log stream metadata store
- this.writerStreamMetadataStore =
- new ZKLogStreamMetadataStore(
- clientId,
- conf,
- sharedWriterZKCForDL,
- scheduler,
- statsLogger);
- this.readerStreamMetadataStore =
- new ZKLogStreamMetadataStore(
- clientId,
- conf,
- sharedReaderZKCForDL,
- scheduler,
- statsLogger);
// create a log segment metadata cache
this.logSegmentMetadataCache = new LogSegmentMetadataCache(conf, Ticker.systemTicker());
+ }
- LOG.info("Constructed BK DistributedLogNamespace : clientId = {}, regionId = {}, federated = {}.",
- new Object[] { clientId, regionId, bkdlConfig.isFederatedNamespace() });
+ @Override
+ public NamespaceDriver getNamespaceDriver() {
+ return driver;
}
//
@@ -468,8 +149,8 @@
throws InvalidStreamNameException, IOException {
checkState();
validateName(logName);
- URI uri = FutureUtils.result(metadataStore.createLog(logName));
- FutureUtils.result(writerStreamMetadataStore.getLog(uri, logName, true, true));
+ URI uri = FutureUtils.result(driver.getLogMetadataStore().createLog(logName));
+ FutureUtils.result(driver.getLogStreamMetadataStore(WRITER).getLog(uri, logName, true, true));
}
@Override
@@ -477,17 +158,15 @@
throws InvalidStreamNameException, LogNotFoundException, IOException {
checkState();
validateName(logName);
- Optional<URI> uri = FutureUtils.result(metadataStore.getLogLocation(logName));
+ Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
if (!uri.isPresent()) {
throw new LogNotFoundException("Log " + logName + " isn't found.");
}
- DistributedLogManager dlm = createDistributedLogManager(
+ DistributedLogManager dlm = openLogInternal(
uri.get(),
logName,
- ClientSharingOption.SharedClients,
Optional.<DistributedLogConfiguration>absent(),
- Optional.<DynamicDistributedLogConfiguration>absent(),
- Optional.<StatsLogger>absent());
+ Optional.<DynamicDistributedLogConfiguration>absent());
dlm.delete();
}
@@ -508,27 +187,26 @@
throws InvalidStreamNameException, IOException {
checkState();
validateName(logName);
- Optional<URI> uri = FutureUtils.result(metadataStore.getLogLocation(logName));
+ Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
if (!uri.isPresent()) {
throw new LogNotFoundException("Log " + logName + " isn't found.");
}
- return createDistributedLogManager(
+ return openLogInternal(
uri.get(),
logName,
- ClientSharingOption.SharedClients,
logConf,
- dynamicLogConf,
- perStreamStatsLogger);
+ dynamicLogConf);
}
@Override
public boolean logExists(String logName)
throws IOException, IllegalArgumentException {
checkState();
- Optional<URI> uri = FutureUtils.result(metadataStore.getLogLocation(logName));
+ Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
if (uri.isPresent()) {
try {
- FutureUtils.result(writerStreamMetadataStore.logExists(uri.get(), logName));
+ FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+ .logExists(uri.get(), logName));
return true;
} catch (LogNotFoundException lnfe) {
return false;
@@ -541,240 +219,18 @@
@Override
public Iterator<String> getLogs() throws IOException {
checkState();
- return FutureUtils.result(metadataStore.getLogs());
+ return FutureUtils.result(driver.getLogMetadataStore().getLogs());
}
@Override
public void registerNamespaceListener(NamespaceListener listener) {
- metadataStore.registerNamespaceListener(listener);
+ driver.getLogMetadataStore().registerNamespaceListener(listener);
}
@Override
public synchronized AccessControlManager createAccessControlManager() throws IOException {
checkState();
- if (null == accessControlManager) {
- String aclRootPath = bkdlConfig.getACLRootPath();
- // Build the access control manager
- if (aclRootPath == null) {
- accessControlManager = DefaultAccessControlManager.INSTANCE;
- LOG.info("Created default access control manager for {}", namespace);
- } else {
- if (!isReservedStreamName(aclRootPath)) {
- throw new IOException("Invalid Access Control List Root Path : " + aclRootPath);
- }
- String zkRootPath = namespace.getPath() + "/" + aclRootPath;
- LOG.info("Creating zk based access control manager @ {} for {}",
- zkRootPath, namespace);
- accessControlManager = new ZKAccessControlManager(conf, sharedReaderZKCForDL,
- zkRootPath, scheduler);
- LOG.info("Created zk based access control manager @ {} for {}",
- zkRootPath, namespace);
- }
- }
- return accessControlManager;
- }
-
- //
- // Legacy methods
- //
-
- 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);
- throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
- }
- String poolName = conf.getLedgerAllocatorPoolName();
- if (null == poolName) {
- LOG.error("No ledger allocator pool name specified when enabling ledger allocator pool.");
- throw new IOException("No ledger allocator name specified when enabling ledger allocator pool.");
- }
- String rootPath = uri.getPath() + "/" + poolPath + "/" + poolName;
- try {
- PathUtils.validatePath(rootPath);
- } catch (IllegalArgumentException iae) {
- 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;
- }
-
- public static ZooKeeperClientBuilder createDLZKClientBuilder(String zkcName,
- DistributedLogConfiguration conf,
- String zkServers,
- StatsLogger statsLogger) {
- RetryPolicy retryPolicy = null;
- if (conf.getZKNumRetries() > 0) {
- retryPolicy = new BoundExponentialBackoffRetryPolicy(
- conf.getZKRetryBackoffStartMillis(),
- conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
- }
- ZooKeeperClientBuilder builder = ZooKeeperClientBuilder.newBuilder()
- .name(zkcName)
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .retryThreadCount(conf.getZKClientNumberRetryThreads())
- .requestRateLimit(conf.getZKRequestRateLimit())
- .zkServers(zkServers)
- .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() });
- return builder;
- }
-
- private static ZooKeeperClientBuilder createBKZKClientBuilder(String zkcName,
- DistributedLogConfiguration conf,
- String zkServers,
- StatsLogger statsLogger) {
- RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
- conf.getBKClientZKRetryBackoffStartMillis(),
- conf.getBKClientZKRetryBackoffMaxMillis(),
- conf.getBKClientZKNumRetries());
- ZooKeeperClientBuilder builder = ZooKeeperClientBuilder.newBuilder()
- .name(zkcName)
- .sessionTimeoutMs(conf.getBKClientZKSessionTimeoutMilliSeconds())
- .retryThreadCount(conf.getZKClientNumberRetryThreads())
- .requestRateLimit(conf.getBKClientZKRequestRateLimit())
- .zkServers(zkServers)
- .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.getBKClientZKNumRetries(),
- conf.getBKClientZKSessionTimeoutMilliSeconds(), conf.getBKClientZKRetryBackoffStartMillis(),
- conf.getBKClientZKRetryBackoffMaxMillis(), conf.getZkAclId() });
- return builder;
- }
-
- private BookKeeperClientBuilder createBKCBuilder(String bkcName,
- DistributedLogConfiguration conf,
- String zkServers,
- String ledgersPath,
- Optional<FeatureProvider> featureProviderOptional) {
- BookKeeperClientBuilder builder = BookKeeperClientBuilder.newBuilder()
- .name(bkcName)
- .dlConfig(conf)
- .zkServers(zkServers)
- .ledgersPath(ledgersPath)
- .channelFactory(channelFactory)
- .requestTimer(requestTimer)
- .featureProvider(featureProviderOptional)
- .statsLogger(statsLogger);
- LOG.info("Created shared client builder {} : zkServers = {}, ledgersPath = {}, numIOThreads = {}",
- new Object[] { bkcName, zkServers, ledgersPath, conf.getBKClientNumberIOThreads() });
- return builder;
- }
-
- @VisibleForTesting
- public ZooKeeperClient getSharedWriterZKCForDL() {
- return sharedWriterZKCForDL;
- }
-
- @VisibleForTesting
- public BookKeeperClient getReaderBKC() {
- return readerBKC;
- }
-
- @VisibleForTesting
- public LogStreamMetadataStore getWriterStreamMetadataStore() {
- return writerStreamMetadataStore;
- }
-
- @VisibleForTesting
- public LedgerAllocator getLedgerAllocator() {
- return allocator;
- }
-
- /**
- * Run given <i>handler</i> by providing an available zookeeper client.
- *
- * @param handler
- * Handler to process with provided zookeeper client.
- * @return result processed by handler.
- * @throws IOException
- */
- private <T> T withZooKeeperClient(ZooKeeperClientHandler<T> handler) throws IOException {
- checkState();
- return handler.handle(sharedWriterZKCForDL);
- }
-
- /**
- * Create a DistributedLogManager for <i>nameOfLogStream</i>, with default shared clients.
- *
- * @param nameOfLogStream
- * name of log stream
- * @return distributedlog manager
- * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
- * @throws IOException
- */
- public DistributedLogManager createDistributedLogManagerWithSharedClients(String nameOfLogStream)
- throws InvalidStreamNameException, IOException {
- return createDistributedLogManager(nameOfLogStream, ClientSharingOption.SharedClients);
- }
-
- /**
- * Create a DistributedLogManager for <i>nameOfLogStream</i>, with specified client sharing options.
- *
- * @param nameOfLogStream
- * name of log stream.
- * @param clientSharingOption
- * specifies if the ZK/BK clients are shared
- * @return distributedlog manager instance.
- * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
- * @throws IOException
- */
- public DistributedLogManager createDistributedLogManager(
- String nameOfLogStream,
- ClientSharingOption clientSharingOption)
- throws InvalidStreamNameException, IOException {
- Optional<DistributedLogConfiguration> logConfiguration = Optional.absent();
- Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration = Optional.absent();
- return createDistributedLogManager(
- nameOfLogStream,
- clientSharingOption,
- logConfiguration,
- dynamicLogConfiguration);
- }
-
- /**
- * Create a DistributedLogManager for <i>nameOfLogStream</i>, with specified client sharing options.
- * Override whitelisted stream-level configuration settings with settings found in
- * <i>logConfiguration</i>.
- *
- *
- * @param nameOfLogStream
- * name of log stream.
- * @param clientSharingOption
- * specifies if the ZK/BK clients are shared
- * @param logConfiguration
- * stream configuration overrides.
- * @param dynamicLogConfiguration
- * dynamic stream configuration overrides.
- * @return distributedlog manager instance.
- * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
- * @throws IOException
- */
- public DistributedLogManager createDistributedLogManager(
- String nameOfLogStream,
- ClientSharingOption clientSharingOption,
- Optional<DistributedLogConfiguration> logConfiguration,
- Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration)
- throws InvalidStreamNameException, IOException {
- if (bkdlConfig.isFederatedNamespace()) {
- throw new UnsupportedOperationException("Use DistributedLogNamespace methods for federated namespace");
- }
- return createDistributedLogManager(
- namespace,
- nameOfLogStream,
- clientSharingOption,
- logConfiguration,
- dynamicLogConfiguration,
- Optional.<StatsLogger>absent()
- );
+ return driver.getAccessControlManager();
}
/**
@@ -784,8 +240,6 @@
* location to store the log
* @param nameOfLogStream
* name of the log
- * @param clientSharingOption
- * client sharing option
* @param logConfiguration
* optional stream configuration
* @param dynamicLogConfiguration
@@ -794,13 +248,11 @@
* @throws InvalidStreamNameException if the stream name is invalid
* @throws IOException
*/
- protected DistributedLogManager createDistributedLogManager(
+ protected DistributedLogManager openLogInternal(
URI uri,
String nameOfLogStream,
- ClientSharingOption clientSharingOption,
Optional<DistributedLogConfiguration> logConfiguration,
- Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration,
- Optional<StatsLogger> perStreamStatsLogger)
+ Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration)
throws InvalidStreamNameException, IOException {
// Make sure the name is well formed
checkState();
@@ -817,172 +269,34 @@
dynConf = ConfUtils.getConstDynConf(mergedConfiguration);
}
- ZooKeeperClientBuilder writerZKCBuilderForDL = null;
- ZooKeeperClientBuilder readerZKCBuilderForDL = null;
- ZooKeeperClient writerZKCForBK = null;
- ZooKeeperClient readerZKCForBK = null;
- BookKeeperClientBuilder writerBKCBuilder = null;
- BookKeeperClientBuilder readerBKCBuilder = null;
-
- switch(clientSharingOption) {
- case SharedClients:
- writerZKCBuilderForDL = sharedWriterZKCBuilderForDL;
- readerZKCBuilderForDL = sharedReaderZKCBuilderForDL;
- writerBKCBuilder = sharedWriterBKCBuilder;
- readerBKCBuilder = sharedReaderBKCBuilder;
- break;
- case SharedZKClientPerStreamBKClient:
- writerZKCBuilderForDL = sharedWriterZKCBuilderForDL;
- readerZKCBuilderForDL = sharedReaderZKCBuilderForDL;
- synchronized (this) {
- if (null == this.sharedWriterZKCForBK) {
- this.sharedWriterZKCBuilderForBK = createBKZKClientBuilder(
- String.format("bkzk:%s:factory_writer_shared", uri),
- mergedConfiguration,
- bkdlConfig.getBkZkServersForWriter(),
- statsLogger.scope("bkzk_factory_writer_shared"));
- this.sharedWriterZKCForBK = this.sharedWriterZKCBuilderForBK.build();
- }
- if (null == this.sharedReaderZKCForBK) {
- if (bkdlConfig.getBkZkServersForWriter().equals(bkdlConfig.getBkZkServersForReader())) {
- this.sharedReaderZKCBuilderForBK = this.sharedWriterZKCBuilderForBK;
- } else {
- this.sharedReaderZKCBuilderForBK = createBKZKClientBuilder(
- String.format("bkzk:%s:factory_reader_shared", uri),
- mergedConfiguration,
- bkdlConfig.getBkZkServersForReader(),
- statsLogger.scope("bkzk_factory_reader_shared"));
- }
- this.sharedReaderZKCForBK = this.sharedReaderZKCBuilderForBK.build();
- }
- writerZKCForBK = this.sharedWriterZKCForBK;
- readerZKCForBK = this.sharedReaderZKCForBK;
- }
- break;
- }
-
- LedgerAllocator dlmLedgerAlloctor = null;
- if (ClientSharingOption.SharedClients == clientSharingOption) {
- dlmLedgerAlloctor = this.allocator;
- }
- // if there's a specified perStreamStatsLogger, user it, otherwise use the default one.
- StatsLogger perLogStatsLogger = perStreamStatsLogger.or(this.perLogStatsLogger);
-
return new BKDistributedLogManager(
nameOfLogStream, /* Log Name */
mergedConfiguration, /* Configuration */
dynConf, /* Dynamic Configuration */
- uri, /* Namespace */
- writerZKCBuilderForDL, /* ZKC Builder for DL Writer */
- readerZKCBuilderForDL, /* ZKC Builder for DL Reader */
- writerZKCForBK, /* ZKC for BookKeeper for DL Writers */
- readerZKCForBK, /* ZKC for BookKeeper for DL Readers */
- writerBKCBuilder, /* BookKeeper Builder for DL Writers */
- readerBKCBuilder, /* BookKeeper Builder for DL Readers */
- writerStreamMetadataStore, /* Log Segment Metadata Store for DL Writers */
- readerStreamMetadataStore, /* Log Segment Metadata Store for DL Readers */
+ uri, /* Namespace URI */
+ driver, /* Namespace Driver */
logSegmentMetadataCache, /* Log Segment Metadata Cache */
scheduler, /* DL scheduler */
- readAheadExecutor, /* Read Aheader Executor */
- channelFactory, /* Netty Channel Factory */
- requestTimer, /* Request Timer */
clientId, /* Client Id */
regionId, /* Region Id */
- dlmLedgerAlloctor, /* Ledger Allocator */
writeLimiter, /* Write Limiter */
featureProvider.scope("dl"), /* Feature Provider */
+ failureInjector, /* Failure Injector */
statsLogger, /* Stats Logger */
- perLogStatsLogger /* Per Log Stats Logger */
+ perLogStatsLogger, /* Per Log Stats Logger */
+ Optional.<AsyncCloseable>absent() /* shared resources, we don't need to close any resources in dlm */
);
}
- public MetadataAccessor createMetadataAccessor(String nameOfMetadataNode)
- throws InvalidStreamNameException, IOException {
- if (bkdlConfig.isFederatedNamespace()) {
- throw new UnsupportedOperationException("Use DistributedLogNamespace methods for federated namespace");
- }
- checkState();
- validateName(nameOfMetadataNode);
- return new ZKMetadataAccessor(nameOfMetadataNode, conf, namespace,
- sharedWriterZKCBuilderForDL, sharedReaderZKCBuilderForDL, statsLogger);
- }
-
- public Collection<String> enumerateAllLogsInNamespace()
- throws IOException, IllegalArgumentException {
- if (bkdlConfig.isFederatedNamespace()) {
- throw new UnsupportedOperationException("Use DistributedLogNamespace methods for federated namespace");
- }
- return Sets.newHashSet(getLogs());
- }
-
- public Map<String, byte[]> enumerateLogsWithMetadataInNamespace()
- throws IOException, IllegalArgumentException {
- if (bkdlConfig.isFederatedNamespace()) {
- throw new UnsupportedOperationException("Use DistributedLogNamespace methods for federated namespace");
- }
- return withZooKeeperClient(new ZooKeeperClientHandler<Map<String, byte[]>>() {
- @Override
- public Map<String, byte[]> handle(ZooKeeperClient zkc) throws IOException {
- return enumerateLogsWithMetadataInternal(zkc, conf, namespace);
- }
- });
- }
-
- private static void validateInput(DistributedLogConfiguration conf, URI uri, String nameOfStream)
- throws IllegalArgumentException, InvalidStreamNameException {
- validateConfAndURI(conf, uri);
- validateName(nameOfStream);
- }
-
- public static Map<String, byte[]> enumerateLogsWithMetadataInNamespace(final DistributedLogConfiguration conf, final URI uri)
- throws IOException, IllegalArgumentException {
- return withZooKeeperClient(new ZooKeeperClientHandler<Map<String, byte[]>>() {
- @Override
- public Map<String, byte[]> handle(ZooKeeperClient zkc) throws IOException {
- return enumerateLogsWithMetadataInternal(zkc, conf, uri);
- }
- }, conf, uri);
- }
-
- private static Map<String, byte[]> enumerateLogsWithMetadataInternal(ZooKeeperClient zkc,
- DistributedLogConfiguration conf, URI uri)
- throws IOException, IllegalArgumentException {
- validateConfAndURI(conf, uri);
- String namespaceRootPath = uri.getPath();
- HashMap<String, byte[]> result = new HashMap<String, byte[]>();
- try {
- ZooKeeper zk = Utils.sync(zkc, namespaceRootPath);
- Stat currentStat = zk.exists(namespaceRootPath, false);
- if (currentStat == null) {
- return result;
- }
- List<String> children = zk.getChildren(namespaceRootPath, false);
- for(String child: children) {
- if (isReservedStreamName(child)) {
- continue;
- }
- String zkPath = String.format("%s/%s", namespaceRootPath, child);
- currentStat = zk.exists(zkPath, false);
- if (currentStat == null) {
- result.put(child, new byte[0]);
- } else {
- result.put(child, zk.getData(zkPath, false, currentStat));
- }
- }
- } catch (InterruptedException ie) {
- LOG.error("Interrupted while deleting " + namespaceRootPath, ie);
- throw new IOException("Interrupted while reading " + namespaceRootPath, ie);
- } catch (KeeperException ke) {
- LOG.error("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
- throw new IOException("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
- }
- return result;
- }
-
+ /**
+ * Check the namespace state.
+ *
+ * @throws IOException
+ */
private void checkState() throws IOException {
if (closed.get()) {
- LOG.error("BKDistributedLogNamespace {} is already closed", namespace);
- throw new AlreadyClosedException("Namespace " + namespace + " is already closed");
+ LOG.error("BK namespace {} is already closed", namespace);
+ throw new AlreadyClosedException("BK namespace " + namespace + " is already closed");
}
}
@@ -991,58 +305,16 @@
*/
@Override
public void close() {
- ZooKeeperClient writerZKC;
- ZooKeeperClient readerZKC;
- AccessControlManager acm;
- if (closed.compareAndSet(false, true)) {
- writerZKC = sharedWriterZKCForBK;
- readerZKC = sharedReaderZKCForBK;
- acm = accessControlManager;
- } else {
+ if (!closed.compareAndSet(false, true)) {
return;
}
- if (null != acm) {
- acm.close();
- LOG.info("Access Control Manager Stopped.");
- }
-
- // Close the allocator
- if (null != allocator) {
- Utils.closeQuietly(allocator);
- LOG.info("Ledger Allocator stopped.");
- }
-
+ // shutdown the driver
+ Utils.close(driver);
+ // close the write limiter
this.writeLimiter.close();
-
- // Shutdown log segment metadata stores
- Utils.close(writerStreamMetadataStore);
- Utils.close(readerStreamMetadataStore);
-
// Shutdown the schedulers
SchedulerUtils.shutdownScheduler(scheduler, conf.getSchedulerShutdownTimeoutMs(),
- TimeUnit.MILLISECONDS);
- LOG.info("Executor Service Stopped.");
- if (scheduler != readAheadExecutor) {
- SchedulerUtils.shutdownScheduler(readAheadExecutor, conf.getSchedulerShutdownTimeoutMs(),
TimeUnit.MILLISECONDS);
- LOG.info("ReadAhead Executor Service Stopped.");
- }
-
- writerBKC.close();
- readerBKC.close();
- sharedWriterZKCForDL.close();
- sharedReaderZKCForDL.close();
-
- // Close shared zookeeper clients for bk
- if (null != writerZKC) {
- writerZKC.close();
- }
- if (null != readerZKC) {
- readerZKC.close();
- }
- channelFactory.releaseExternalResources();
- LOG.info("Release external resources used by channel factory.");
- requestTimer.stop();
- LOG.info("Stopped request timer");
+ LOG.info("Executor Service Stopped.");
}
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
index 25b25e2..2486297 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
@@ -20,7 +20,6 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Stopwatch;
import com.google.common.collect.Lists;
-import com.twitter.distributedlog.bk.LedgerAllocator;
import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
import com.twitter.distributedlog.exceptions.DLIllegalStateException;
import com.twitter.distributedlog.exceptions.EndOfStreamException;
@@ -29,8 +28,8 @@
import com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException;
import com.twitter.distributedlog.exceptions.UnexpectedException;
import com.twitter.distributedlog.function.GetLastTxIdFunction;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
import com.twitter.distributedlog.metadata.LogMetadataForWriter;
import com.twitter.distributedlog.lock.DistributedLock;
import com.twitter.distributedlog.logsegment.LogSegmentFilter;
@@ -41,6 +40,7 @@
import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
import com.twitter.distributedlog.metadata.MetadataUpdater;
import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
+import com.twitter.distributedlog.util.Allocator;
import com.twitter.distributedlog.util.DLUtils;
import com.twitter.distributedlog.util.FailpointUtils;
import com.twitter.distributedlog.util.FutureUtils;
@@ -53,7 +53,6 @@
import com.twitter.util.Future;
import com.twitter.util.FutureEventListener;
import com.twitter.util.Promise;
-import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.feature.FeatureProvider;
import org.apache.bookkeeper.stats.AlertStatsLogger;
import org.apache.bookkeeper.stats.OpStatsLogger;
@@ -88,9 +87,22 @@
class BKLogWriteHandler extends BKLogHandler {
static final Logger LOG = LoggerFactory.getLogger(BKLogReadHandler.class);
+ private static Transaction.OpListener<LogSegmentEntryWriter> NULL_OP_LISTENER =
+ new Transaction.OpListener<LogSegmentEntryWriter>() {
+ @Override
+ public void onCommit(LogSegmentEntryWriter r) {
+ // no-op
+ }
+
+ @Override
+ public void onAbort(Throwable t) {
+ // no-op
+ }
+ };
+
protected final LogMetadataForWriter logMetadataForWriter;
+ protected final Allocator<LogSegmentEntryWriter, Object> logSegmentAllocator;
protected final DistributedLock lock;
- protected final LedgerAllocator ledgerAllocator;
protected final MaxTxId maxTxId;
protected final MaxLogSegmentSequenceNo maxLogSegmentSequenceNo;
protected final boolean validateLogSegmentSequenceNumber;
@@ -154,7 +166,7 @@
LogSegmentMetadataCache metadataCache,
LogSegmentEntryStore entryStore,
OrderedScheduler scheduler,
- LedgerAllocator allocator,
+ Allocator<LogSegmentEntryWriter, Object> segmentAllocator,
StatsLogger statsLogger,
StatsLogger perLogStatsLogger,
AlertStatsLogger alertStatsLogger,
@@ -174,11 +186,11 @@
alertStatsLogger,
clientId);
this.logMetadataForWriter = logMetadata;
+ this.logSegmentAllocator = segmentAllocator;
this.perLogStatsLogger = perLogStatsLogger;
this.writeLimiter = writeLimiter;
this.featureProvider = featureProvider;
this.dynConf = dynConf;
- this.ledgerAllocator = allocator;
this.lock = lock;
this.metadataUpdater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
@@ -523,7 +535,7 @@
}
try {
- ledgerAllocator.allocate();
+ logSegmentAllocator.allocate();
} catch (IOException e) {
// failed to issue an allocation request
failStartLogSegment(promise, bestEffort, e);
@@ -541,25 +553,16 @@
return;
}
- ledgerAllocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
- @Override
- public void onCommit(LedgerHandle lh) {
- // no-op
- }
+ logSegmentAllocator.tryObtain(txn, NULL_OP_LISTENER)
+ .addEventListener(new FutureEventListener<LogSegmentEntryWriter>() {
@Override
- public void onAbort(Throwable t) {
- // no-op
- }
- }).addEventListener(new FutureEventListener<LedgerHandle>() {
-
- @Override
- public void onSuccess(LedgerHandle lh) {
+ public void onSuccess(LogSegmentEntryWriter entryWriter) {
// try-obtain succeed
createInprogressLogSegment(
txn,
txId,
- lh,
+ entryWriter,
bestEffort,
promise);
}
@@ -586,7 +589,7 @@
// just leak from the allocation pool - hence cause "No Ledger Allocator"
private void createInprogressLogSegment(Transaction<Object> txn,
final long txId,
- final LedgerHandle lh,
+ final LogSegmentEntryWriter entryWriter,
boolean bestEffort,
final Promise<BKLogSegmentWriter> promise) {
final long logSegmentSeqNo;
@@ -601,13 +604,15 @@
return;
}
- final String inprogressZnodePath = inprogressZNode(lh.getId(), txId, logSegmentSeqNo);
+ final String inprogressZnodePath = inprogressZNode(
+ entryWriter.getLogSegmentId(), txId, logSegmentSeqNo);
final LogSegmentMetadata l =
new LogSegmentMetadata.LogSegmentMetadataBuilder(inprogressZnodePath,
- conf.getDLLedgerMetadataLayoutVersion(), lh.getId(), txId)
+ conf.getDLLedgerMetadataLayoutVersion(), entryWriter.getLogSegmentId(), txId)
.setLogSegmentSequenceNo(logSegmentSeqNo)
.setRegionId(regionId)
- .setEnvelopeEntries(LogSegmentMetadata.supportsEnvelopedEntries(conf.getDLLedgerMetadataLayoutVersion()))
+ .setEnvelopeEntries(
+ LogSegmentMetadata.supportsEnvelopedEntries(conf.getDLLedgerMetadataLayoutVersion()))
.build();
// Create an inprogress segment
@@ -631,7 +636,7 @@
l.getSegmentName(),
conf,
conf.getDLLedgerMetadataLayoutVersion(),
- new BKLogSegmentEntryWriter(lh),
+ entryWriter,
lock,
txId,
logSegmentSeqNo,
@@ -1268,8 +1273,7 @@
public Future<Void> asyncClose() {
return Utils.closeSequence(scheduler,
lock,
- ledgerAllocator
- );
+ logSegmentAllocator);
}
@Override
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
index c39ae4c..8d3c418 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
@@ -250,15 +250,22 @@
return promise;
}
- public synchronized void close() {
- if (closed) {
- return;
+ public void close() {
+ BookKeeper bkcToClose;
+ ZooKeeperClient zkcToClose;
+ synchronized (this) {
+ if (closed) {
+ return;
+ }
+ closed = true;
+ bkcToClose = bkc;
+ zkcToClose = zkc;
}
LOG.info("BookKeeper Client closed {}", name);
- if (null != bkc) {
+ if (null != bkcToClose) {
try {
- bkc.close();
+ bkcToClose.close();
} catch (InterruptedException e) {
LOG.warn("Interrupted on closing bookkeeper client {} : ", name, e);
Thread.currentThread().interrupt();
@@ -266,12 +273,11 @@
LOG.warn("Error on closing bookkeeper client {} : ", name, e);
}
}
- if (null != zkc) {
+ if (null != zkcToClose) {
if (ownZK) {
- zkc.close();
+ zkcToClose.close();
}
}
- closed = true;
}
public synchronized void checkClosedOrInError() throws AlreadyClosedException {
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
index 6c6017e..6da4b8d 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
@@ -1301,6 +1301,7 @@
* @return number of dedicated readahead worker threads.
* @see #getNumWorkerThreads()
*/
+ @Deprecated
public int getNumReadAheadWorkerThreads() {
return getInt(BKDL_NUM_READAHEAD_WORKER_THREADS, 0);
}
@@ -1313,6 +1314,7 @@
* @return configuration
* @see #getNumReadAheadWorkerThreads()
*/
+ @Deprecated
public DistributedLogConfiguration setNumReadAheadWorkerThreads(int numWorkerThreads) {
setProperty(BKDL_NUM_READAHEAD_WORKER_THREADS, numWorkerThreads);
return this;
@@ -3515,8 +3517,11 @@
Preconditions.checkArgument(getBKClientReadTimeout() * 1000 >= getReadLACLongPollTimeout(),
"Invalid timeout configuration: bkcReadTimeoutSeconds ("+getBKClientReadTimeout()+
") should be longer than readLACLongPollTimeout ("+getReadLACLongPollTimeout()+")");
- Preconditions.checkArgument(getReaderIdleWarnThresholdMillis() > 2 * getReadLACLongPollTimeout(),
- "Invalid configuration: ReaderIdleWarnThreshold should be 2x larget than readLACLongPollTimeout");
+ long readerIdleWarnThresholdMs = getReaderIdleWarnThresholdMillis();
+ if (readerIdleWarnThresholdMs > 0) { // NOTE: some test cases set the idle warn threshold to 0
+ Preconditions.checkArgument(readerIdleWarnThresholdMs > 2 * getReadLACLongPollTimeout(),
+ "Invalid configuration: ReaderIdleWarnThreshold should be 2x larget than readLACLongPollTimeout");
+ }
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
index ccb0778..34cfb65 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
@@ -18,9 +18,13 @@
package com.twitter.distributedlog;
import com.twitter.distributedlog.callback.LogSegmentListener;
+import com.twitter.distributedlog.io.AsyncCloseable;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
import com.twitter.distributedlog.subscription.SubscriptionStateStore;
import com.twitter.distributedlog.subscription.SubscriptionsStore;
import com.twitter.util.Future;
+
+import java.io.Closeable;
import java.io.IOException;
import java.util.List;
@@ -31,7 +35,20 @@
* each conceptual place of storage corresponds to exactly one instance of
* this class, which is created when the EditLog is first opened.
*/
-public interface DistributedLogManager extends MetadataAccessor {
+public interface DistributedLogManager extends AsyncCloseable, Closeable {
+
+ /**
+ * Get the name of the stream managed by this log manager
+ * @return streamName
+ */
+ public String getStreamName();
+
+ /**
+ * Get the namespace driver used by this manager.
+ *
+ * @return the namespace driver
+ */
+ public NamespaceDriver getNamespaceDriver();
/**
* Get log segments.
@@ -282,15 +299,6 @@
public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException;
/**
- * Get the subscription state storage provided by the distributed log manager
- *
- * @param subscriberId - Application specific Id associated with the subscriber
- * @return Subscription state store
- */
- @Deprecated
- public SubscriptionStateStore getSubscriptionStateStore(String subscriberId);
-
- /**
* Get the subscriptions store provided by the distributedlog manager.
*
* @return subscriptions store manages subscriptions for current stream.
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManagerFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManagerFactory.java
deleted file mode 100644
index 4caeeba..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManagerFactory.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/**
- * 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.
- */
-package com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Collection;
-import java.util.Map;
-
-/**
- * This is the legacy way to access bookkeeper based distributedlog namespace.
- * Use {@link DistributedLogNamespace} to manage logs instead if you could.
- */
-@Deprecated
-public class DistributedLogManagerFactory {
- static final Logger LOG = LoggerFactory.getLogger(DistributedLogManagerFactory.class);
-
- public static enum ClientSharingOption {
- PerStreamClients,
- SharedZKClientPerStreamBKClient,
- SharedClients
- }
-
- private final BKDistributedLogNamespace namespace;
-
- public DistributedLogManagerFactory(DistributedLogConfiguration conf, URI uri)
- throws IOException, IllegalArgumentException {
- this(conf, uri, NullStatsLogger.INSTANCE);
- }
-
- public DistributedLogManagerFactory(DistributedLogConfiguration conf, URI uri,
- StatsLogger statsLogger)
- throws IOException, IllegalArgumentException {
- this(conf,
- uri,
- statsLogger,
- DistributedLogConstants.UNKNOWN_CLIENT_ID,
- DistributedLogConstants.LOCAL_REGION_ID);
- }
-
- public DistributedLogManagerFactory(DistributedLogConfiguration conf,
- URI uri,
- StatsLogger statsLogger,
- String clientId,
- int regionId)
- throws IOException, IllegalArgumentException {
- this.namespace = BKDistributedLogNamespace.newBuilder()
- .conf(conf)
- .uri(uri)
- .statsLogger(statsLogger)
- .clientId(clientId)
- .regionId(regionId)
- .build();
- }
-
- public DistributedLogNamespace getNamespace() {
- return namespace;
- }
-
- public void registerNamespaceListener(NamespaceListener listener) {
- namespace.registerNamespaceListener(listener);
- }
-
- /**
- * Create a DistributedLogManager for <i>nameOfLogStream</i>, with default shared clients.
- *
- * @param nameOfLogStream
- * name of log stream
- * @return distributedlog manager
- * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
- * @throws IOException
- */
- public DistributedLogManager createDistributedLogManagerWithSharedClients(String nameOfLogStream)
- throws InvalidStreamNameException, IOException {
- return createDistributedLogManager(nameOfLogStream, ClientSharingOption.SharedClients);
- }
-
- /**
- * Create a DistributedLogManager for <i>nameOfLogStream</i>, with specified client sharing options.
- *
- * @param nameOfLogStream
- * name of log stream.
- * @param clientSharingOption
- * specifies if the ZK/BK clients are shared
- * @return distributedlog manager instance.
- * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
- * @throws IOException
- */
- public DistributedLogManager createDistributedLogManager(
- String nameOfLogStream,
- ClientSharingOption clientSharingOption)
- throws InvalidStreamNameException, IOException {
- Optional<DistributedLogConfiguration> streamConfiguration = Optional.absent();
- Optional<DynamicDistributedLogConfiguration> dynamicStreamConfiguration = Optional.absent();
- return createDistributedLogManager(nameOfLogStream,
- clientSharingOption,
- streamConfiguration,
- dynamicStreamConfiguration);
- }
-
- /**
- * Create a DistributedLogManager for <i>nameOfLogStream</i>, with specified client sharing options.
- * This method allows the caller to override global configuration options by supplying stream
- * configuration overrides. Stream config overrides come in two flavors, static and dynamic. Static
- * config never changes, and DynamicDistributedLogConfiguration is a) reloaded periodically and
- * b) safe to access from any context.
- *
- * @param nameOfLogStream
- * name of log stream.
- * @param clientSharingOption
- * specifies if the ZK/BK clients are shared
- * @param streamConfiguration
- * stream configuration overrides.
- * @param dynamicStreamConfiguration
- * dynamic stream configuration overrides.
- * @return distributedlog manager instance.
- * @throws com.twitter.distributedlog.exceptions.InvalidStreamNameException if stream name is invalid
- * @throws IOException
- */
- public DistributedLogManager createDistributedLogManager(
- String nameOfLogStream,
- ClientSharingOption clientSharingOption,
- Optional<DistributedLogConfiguration> streamConfiguration,
- Optional<DynamicDistributedLogConfiguration> dynamicStreamConfiguration)
- throws InvalidStreamNameException, IOException {
- return namespace.createDistributedLogManager(
- nameOfLogStream,
- clientSharingOption,
- streamConfiguration,
- dynamicStreamConfiguration);
- }
-
- public MetadataAccessor createMetadataAccessor(String nameOfMetadataNode)
- throws InvalidStreamNameException, IOException {
- return namespace.createMetadataAccessor(nameOfMetadataNode);
- }
-
- public synchronized AccessControlManager createAccessControlManager() throws IOException {
- return namespace.createAccessControlManager();
- }
-
- public boolean checkIfLogExists(String nameOfLogStream)
- throws IOException, IllegalArgumentException {
- return namespace.logExists(nameOfLogStream);
- }
-
- public Collection<String> enumerateAllLogsInNamespace()
- throws IOException, IllegalArgumentException {
- return namespace.enumerateAllLogsInNamespace();
- }
-
- public Map<String, byte[]> enumerateLogsWithMetadataInNamespace()
- throws IOException, IllegalArgumentException {
- return namespace.enumerateLogsWithMetadataInNamespace();
- }
-
- /**
- * This method is to initialize the metadata for a unpartitioned stream with name <i>streamName</i>.
- *
- * TODO: after 0.2 is upgraded to 0.3, remove this.
- *
- * @param streamName
- * stream name.
- * @throws IOException
- */
- public void createUnpartitionedStream(final String streamName) throws IOException {
- namespace.createLog(streamName);
- }
-
- /**
- * Close the distributed log manager factory, freeing any resources it may hold.
- */
- public void close() {
- namespace.close();
- }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
index 85a370f..f4a1e41 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
@@ -18,7 +18,7 @@
package com.twitter.distributedlog;
import com.google.common.base.Optional;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import com.twitter.distributedlog.metadata.DLMetadata;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.proto.BookieServer;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
index 94e618a..19f4497 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
@@ -65,6 +65,7 @@
public class ReadAheadEntryReader implements
AsyncCloseable,
LogSegmentListener,
+ LogSegmentEntryReader.StateChangeListener,
FutureEventListener<List<Entry.Reader>> {
private static final Logger logger = LoggerFactory.getLogger(ReadAheadEntryReader.class);
@@ -169,6 +170,9 @@
@Override
synchronized public void onSuccess(LogSegmentEntryReader reader) {
this.reader = reader;
+ if (reader.getSegment().isInProgress()) {
+ reader.registerListener(ReadAheadEntryReader.this);
+ }
}
@Override
@@ -271,7 +275,7 @@
// State of the reader
//
- private boolean isInitialized;
+ private boolean isInitialized = false;
private boolean readAheadPaused = false;
private Promise<Void> closePromise = null;
// segment readers
@@ -549,10 +553,22 @@
}
}
+ void markCaughtup() {
+ if (isCatchingUp) {
+ isCatchingUp = false;
+ logger.info("ReadAhead for {} is caught up", readHandler.getFullyQualifiedName());
+ }
+ }
+
public boolean isReadAheadCaughtUp() {
return !isCatchingUp;
}
+ @Override
+ public void onCaughtupOnInprogress() {
+ markCaughtup();
+ }
+
//
// ReadAhead State Machine
//
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
index 90807b0..15f1805 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
@@ -20,7 +20,7 @@
import com.google.common.base.Preconditions;
import com.twitter.distributedlog.ZooKeeperClient.Credentials;
import com.twitter.distributedlog.ZooKeeperClient.DigestCredentials;
-import com.twitter.distributedlog.util.DLUtils;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.zookeeper.RetryPolicy;
@@ -139,7 +139,7 @@
* @return builder.
*/
public synchronized ZooKeeperClientBuilder uri(URI uri) {
- this.zkServers = DLUtils.getZKServersFromDLUri(uri);
+ this.zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
return this;
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
index 0a3fdb0..0512907 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
@@ -18,28 +18,27 @@
package com.twitter.distributedlog.admin;
import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DistributedLogConfiguration;
+import com.google.common.collect.Lists;
import com.twitter.distributedlog.DistributedLogManager;
import com.twitter.distributedlog.LogRecordWithDLSN;
import com.twitter.distributedlog.LogSegmentMetadata;
import com.twitter.distributedlog.ReadUtils;
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.acl.ZKAccessControl;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
+import com.twitter.distributedlog.impl.acl.ZKAccessControl;
import com.twitter.distributedlog.exceptions.DLIllegalStateException;
import com.twitter.distributedlog.impl.federated.FederatedZKLogMetadataStore;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import com.twitter.distributedlog.metadata.DLMetadata;
import com.twitter.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
import com.twitter.distributedlog.metadata.MetadataUpdater;
import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
+import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
import com.twitter.distributedlog.thrift.AccessControlEntry;
import com.twitter.distributedlog.tools.DistributedLogTool;
-import com.twitter.distributedlog.util.DLUtils;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.OrderedScheduler;
import com.twitter.distributedlog.util.SchedulerUtils;
@@ -61,6 +60,7 @@
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.SortedSet;
@@ -76,7 +76,6 @@
/**
* Admin Tool for DistributedLog.
*/
-@SuppressWarnings("deprecation")
public class DistributedLogAdmin extends DistributedLogTool {
static final Logger LOG = LoggerFactory.getLogger(DistributedLogAdmin.class);
@@ -84,8 +83,8 @@
/**
* Fix inprogress segment with lower ledger sequence number.
*
- * @param factory
- * dlm factory.
+ * @param namespace
+ * dl namespace
* @param metadataUpdater
* metadata updater.
* @param streamName
@@ -96,12 +95,12 @@
* is confirmation needed before executing actual action.
* @throws IOException
*/
- public static void fixInprogressSegmentWithLowerSequenceNumber(final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+ public static void fixInprogressSegmentWithLowerSequenceNumber(final DistributedLogNamespace namespace,
final MetadataUpdater metadataUpdater,
final String streamName,
final boolean verbose,
final boolean interactive) throws IOException {
- DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+ DistributedLogManager dlm = namespace.openLog(streamName);
try {
List<LogSegmentMetadata> segments = dlm.getLogSegments();
if (verbose) {
@@ -194,37 +193,37 @@
}
public static void checkAndRepairDLNamespace(final URI uri,
- final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+ final DistributedLogNamespace namespace,
final MetadataUpdater metadataUpdater,
final OrderedScheduler scheduler,
- final BookKeeperClient bkc,
- final String digestpw,
final boolean verbose,
final boolean interactive) throws IOException {
- checkAndRepairDLNamespace(uri, factory, metadataUpdater, scheduler, bkc, digestpw, verbose, interactive, 1);
+ checkAndRepairDLNamespace(uri, namespace, metadataUpdater, scheduler, verbose, interactive, 1);
}
public static void checkAndRepairDLNamespace(final URI uri,
- final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+ final DistributedLogNamespace namespace,
final MetadataUpdater metadataUpdater,
final OrderedScheduler scheduler,
- final BookKeeperClient bkc,
- final String digestpw,
final boolean verbose,
final boolean interactive,
final int concurrency) throws IOException {
Preconditions.checkArgument(concurrency > 0, "Invalid concurrency " + concurrency + " found.");
// 0. getting streams under a given uri.
- Collection<String> streams = factory.enumerateAllLogsInNamespace();
+ Iterator<String> streamsIter = namespace.getLogs();
+ List<String> streams = Lists.newArrayList();
+ while (streamsIter.hasNext()) {
+ streams.add(streamsIter.next());
+ }
if (verbose) {
- System.out.println("- 0. checking " + streams.size() + " streams under " + uri);
+ System.out.println("- 0. checking streams under " + uri);
}
if (streams.size() == 0) {
System.out.println("+ 0. nothing to check. quit.");
return;
}
Map<String, StreamCandidate> streamCandidates =
- checkStreams(factory, streams, scheduler, bkc, digestpw, concurrency);
+ checkStreams(namespace, streams, scheduler, concurrency);
if (verbose) {
System.out.println("+ 0. " + streamCandidates.size() + " corrupted streams found.");
}
@@ -248,11 +247,9 @@
}
private static Map<String, StreamCandidate> checkStreams(
- final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+ final DistributedLogNamespace namespace,
final Collection<String> streams,
final OrderedScheduler scheduler,
- final BookKeeperClient bkc,
- final String digestpw,
final int concurrency) throws IOException {
final LinkedBlockingQueue<String> streamQueue =
new LinkedBlockingQueue<String>();
@@ -275,7 +272,7 @@
StreamCandidate candidate;
try {
LOG.info("Checking stream {}.", stream);
- candidate = checkStream(factory, stream, scheduler, bkc, digestpw);
+ candidate = checkStream(namespace, stream, scheduler);
LOG.info("Checked stream {} - {}.", stream, candidate);
} catch (IOException e) {
LOG.error("Error on checking stream {} : ", stream, e);
@@ -316,12 +313,10 @@
}
private static StreamCandidate checkStream(
- final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+ final DistributedLogNamespace namespace,
final String streamName,
- final OrderedScheduler scheduler,
- final BookKeeperClient bkc,
- String digestpw) throws IOException {
- DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+ final OrderedScheduler scheduler) throws IOException {
+ DistributedLogManager dlm = namespace.openLog(streamName);
try {
List<LogSegmentMetadata> segments = dlm.getLogSegments();
if (segments.isEmpty()) {
@@ -330,7 +325,7 @@
List<Future<LogSegmentCandidate>> futures =
new ArrayList<Future<LogSegmentCandidate>>(segments.size());
for (LogSegmentMetadata segment : segments) {
- futures.add(checkLogSegment(streamName, segment, scheduler, bkc, digestpw));
+ futures.add(checkLogSegment(namespace, streamName, segment, scheduler));
}
List<LogSegmentCandidate> segmentCandidates;
try {
@@ -354,21 +349,16 @@
}
private static Future<LogSegmentCandidate> checkLogSegment(
+ final DistributedLogNamespace namespace,
final String streamName,
final LogSegmentMetadata metadata,
- final OrderedScheduler scheduler,
- final BookKeeperClient bkc,
- final String digestpw) {
+ final OrderedScheduler scheduler) {
if (metadata.isInProgress()) {
return Future.value(null);
}
- final LogSegmentEntryStore entryStore = new BKLogSegmentEntryStore(
- new DistributedLogConfiguration().setBKDigestPW(digestpw),
- bkc,
- scheduler,
- NullStatsLogger.INSTANCE,
- AsyncFailureInjector.NULL);
+ final LogSegmentEntryStore entryStore = namespace.getNamespaceDriver()
+ .getLogSegmentEntryStore(NamespaceDriver.Role.READER);
return ReadUtils.asyncReadLastRecord(
streamName,
metadata,
@@ -432,6 +422,8 @@
/**
* Unbind the bookkeeper environment for a given distributedlog uri.
+ *
+ * TODO: move unbind operation to namespace driver
*/
class UnbindCommand extends OptsCommand {
@@ -491,6 +483,8 @@
/**
* Bind Command to bind bookkeeper environment for a given distributed uri.
+ *
+ * TODO: move bind to namespace driver
*/
class BindCommand extends OptsCommand {
@@ -559,7 +553,7 @@
if (cmdline.hasOption("dlzw")) {
dlZkServersForWriter = cmdline.getOptionValue("dlzw");
} else {
- dlZkServersForWriter = DLUtils.getZKServersFromDLUri(uri);
+ dlZkServersForWriter = BKNamespaceDriver.getZKServersFromDLUri(uri);
}
if (cmdline.hasOption("dlzr")) {
dlZkServersForReader = cmdline.getOptionValue("dlzr");
@@ -689,7 +683,7 @@
return -1;
}
for (String stream : streams) {
- fixInprogressSegmentWithLowerSequenceNumber(getFactory(), metadataUpdater, stream, verbose, !getForce());
+ fixInprogressSegmentWithLowerSequenceNumber(getNamespace(), metadataUpdater, stream, verbose, !getForce());
}
return 0;
}
@@ -739,10 +733,9 @@
.corePoolSize(Runtime.getRuntime().availableProcessors())
.build();
ExecutorService executorService = Executors.newCachedThreadPool();
- BookKeeperClient bkc = getBookKeeperClient();
try {
- checkAndRepairDLNamespace(getUri(), getFactory(), metadataUpdater, scheduler,
- bkc, getConf().getBKDigestPW(), verbose, !getForce(), concurrency);
+ checkAndRepairDLNamespace(getUri(), getNamespace(), metadataUpdater, scheduler,
+ verbose, !getForce(), concurrency);
} finally {
SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
index e551c22..a081606 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
@@ -21,18 +21,20 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.SettableFuture;
-import com.twitter.distributedlog.BKDistributedLogNamespace;
import com.twitter.distributedlog.BookKeeperClient;
import com.twitter.distributedlog.BookKeeperClientBuilder;
import com.twitter.distributedlog.DistributedLogConfiguration;
import com.twitter.distributedlog.DistributedLogManager;
import com.twitter.distributedlog.LogSegmentMetadata;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.ZooKeeperClientBuilder;
import com.twitter.distributedlog.exceptions.DLInterruptedException;
import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
import com.twitter.distributedlog.util.DLUtils;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
@@ -52,8 +54,8 @@
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.HashSet;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -72,7 +74,6 @@
/**
* DL Auditor will audit DL namespace, e.g. find leaked ledger, report disk usage by streams.
*/
-@SuppressWarnings("deprecation")
public class DLAuditor {
private static final Logger logger = LoggerFactory.getLogger(DLAuditor.class);
@@ -83,23 +84,23 @@
this.conf = conf;
}
- private ZooKeeperClient getZooKeeperClient(com.twitter.distributedlog.DistributedLogManagerFactory factory) {
- DistributedLogNamespace namespace = factory.getNamespace();
- assert(namespace instanceof BKDistributedLogNamespace);
- return ((BKDistributedLogNamespace) namespace).getSharedWriterZKCForDL();
+ private ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) {
+ NamespaceDriver driver = namespace.getNamespaceDriver();
+ assert(driver instanceof BKNamespaceDriver);
+ return ((BKNamespaceDriver) driver).getWriterZKC();
}
- private BookKeeperClient getBookKeeperClient(com.twitter.distributedlog.DistributedLogManagerFactory factory) {
- DistributedLogNamespace namespace = factory.getNamespace();
- assert(namespace instanceof BKDistributedLogNamespace);
- return ((BKDistributedLogNamespace) namespace).getReaderBKC();
+ private BookKeeperClient getBookKeeperClient(DistributedLogNamespace namespace) {
+ NamespaceDriver driver = namespace.getNamespaceDriver();
+ assert(driver instanceof BKNamespaceDriver);
+ return ((BKNamespaceDriver) driver).getReaderBKC();
}
private String validateAndGetZKServers(List<URI> uris) {
URI firstURI = uris.get(0);
- String zkServers = DLUtils.getZKServersFromDLUri(firstURI);
+ String zkServers = BKNamespaceDriver.getZKServersFromDLUri(firstURI);
for (URI uri : uris) {
- if (!zkServers.equalsIgnoreCase(DLUtils.getZKServersFromDLUri(uri))) {
+ if (!zkServers.equalsIgnoreCase(BKNamespaceDriver.getZKServersFromDLUri(uri))) {
throw new IllegalArgumentException("Uris don't belong to same zookeeper cluster");
}
}
@@ -224,19 +225,23 @@
private Set<Long> collectLedgersFromDL(List<URI> uris, List<List<String>> allocationPaths)
throws IOException {
final Set<Long> ledgers = new TreeSet<Long>();
- List<com.twitter.distributedlog.DistributedLogManagerFactory> factories =
- new ArrayList<com.twitter.distributedlog.DistributedLogManagerFactory>(uris.size());
+ List<DistributedLogNamespace> namespaces =
+ new ArrayList<DistributedLogNamespace>(uris.size());
try {
for (URI uri : uris) {
- factories.add(new com.twitter.distributedlog.DistributedLogManagerFactory(conf, uri));
+ namespaces.add(
+ DistributedLogNamespaceBuilder.newBuilder()
+ .conf(conf)
+ .uri(uri)
+ .build());
}
final CountDownLatch doneLatch = new CountDownLatch(uris.size());
final AtomicInteger numFailures = new AtomicInteger(0);
ExecutorService executor = Executors.newFixedThreadPool(uris.size());
try {
int i = 0;
- for (com.twitter.distributedlog.DistributedLogManagerFactory factory : factories) {
- final com.twitter.distributedlog.DistributedLogManagerFactory dlFactory = factory;
+ for (final DistributedLogNamespace namespace : namespaces) {
+ final DistributedLogNamespace dlNamespace = namespace;
final URI uri = uris.get(i);
final List<String> aps = allocationPaths.get(i);
i++;
@@ -245,12 +250,12 @@
public void run() {
try {
logger.info("Collecting ledgers from {} : {}", uri, aps);
- collectLedgersFromAllocator(uri, dlFactory, aps, ledgers);
+ collectLedgersFromAllocator(uri, namespace, aps, ledgers);
synchronized (ledgers) {
logger.info("Collected {} ledgers from allocators for {} : {} ",
new Object[]{ledgers.size(), uri, ledgers});
}
- collectLedgersFromDL(uri, dlFactory, ledgers);
+ collectLedgersFromDL(uri, namespace, ledgers);
} catch (IOException e) {
numFailures.incrementAndGet();
logger.info("Error to collect ledgers from DL : ", e);
@@ -273,15 +278,15 @@
executor.shutdown();
}
} finally {
- for (com.twitter.distributedlog.DistributedLogManagerFactory factory : factories) {
- factory.close();
+ for (DistributedLogNamespace namespace : namespaces) {
+ namespace.close();
}
}
return ledgers;
}
private void collectLedgersFromAllocator(final URI uri,
- final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+ final DistributedLogNamespace namespace,
final List<String> allocationPaths,
final Set<Long> ledgers) throws IOException {
final LinkedBlockingQueue<String> poolQueue =
@@ -289,7 +294,7 @@
for (String allocationPath : allocationPaths) {
String rootPath = uri.getPath() + "/" + allocationPath;
try {
- List<String> pools = getZooKeeperClient(factory).get().getChildren(rootPath, false);
+ List<String> pools = getZooKeeperClient(namespace).get().getChildren(rootPath, false);
for (String pool : pools) {
poolQueue.add(rootPath + "/" + pool);
}
@@ -318,11 +323,11 @@
private void collectLedgersFromPool(String poolPath)
throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
- List<String> allocators = getZooKeeperClient(factory).get()
+ List<String> allocators = getZooKeeperClient(namespace).get()
.getChildren(poolPath, false);
for (String allocator : allocators) {
String allocatorPath = poolPath + "/" + allocator;
- byte[] data = getZooKeeperClient(factory).get().getData(allocatorPath, false, new Stat());
+ byte[] data = getZooKeeperClient(namespace).get().getData(allocatorPath, false, new Stat());
if (null != data && data.length > 0) {
try {
long ledgerId = DLUtils.bytes2LogSegmentId(data);
@@ -341,30 +346,31 @@
}
private void collectLedgersFromDL(final URI uri,
- final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+ final DistributedLogNamespace namespace,
final Set<Long> ledgers) throws IOException {
logger.info("Enumerating {} to collect streams.", uri);
- Collection<String> streams = factory.enumerateAllLogsInNamespace();
+ Iterator<String> streams = namespace.getLogs();
final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
- streamQueue.addAll(streams);
+ while (streams.hasNext()) {
+ streamQueue.add(streams.next());
+ }
logger.info("Collected {} streams from uri {} : {}",
- new Object[] { streams.size(), uri, streams });
+ new Object[] { streamQueue.size(), uri, streams });
executeAction(streamQueue, 10, new Action<String>() {
@Override
public void execute(String stream) throws IOException {
- collectLedgersFromStream(factory, stream, ledgers);
+ collectLedgersFromStream(namespace, stream, ledgers);
}
});
}
- private List<Long> collectLedgersFromStream(com.twitter.distributedlog.DistributedLogManagerFactory factory,
+ private List<Long> collectLedgersFromStream(DistributedLogNamespace namespace,
String stream,
Set<Long> ledgers)
throws IOException {
- DistributedLogManager dlm = factory.createDistributedLogManager(stream,
- com.twitter.distributedlog.DistributedLogManagerFactory.ClientSharingOption.SharedClients);
+ DistributedLogManager dlm = namespace.openLog(stream);
try {
List<LogSegmentMetadata> segments = dlm.getLogSegments();
List<Long> sLedgers = new ArrayList<Long>();
@@ -388,21 +394,25 @@
*/
public Map<String, Long> calculateStreamSpaceUsage(final URI uri) throws IOException {
logger.info("Collecting stream space usage for {}.", uri);
- com.twitter.distributedlog.DistributedLogManagerFactory factory =
- new com.twitter.distributedlog.DistributedLogManagerFactory(conf, uri);
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+ .conf(conf)
+ .uri(uri)
+ .build();
try {
- return calculateStreamSpaceUsage(uri, factory);
+ return calculateStreamSpaceUsage(uri, namespace);
} finally {
- factory.close();
+ namespace.close();
}
}
private Map<String, Long> calculateStreamSpaceUsage(
- final URI uri, final com.twitter.distributedlog.DistributedLogManagerFactory factory)
+ final URI uri, final DistributedLogNamespace namespace)
throws IOException {
- Collection<String> streams = factory.enumerateAllLogsInNamespace();
+ Iterator<String> streams = namespace.getLogs();
final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
- streamQueue.addAll(streams);
+ while (streams.hasNext()) {
+ streamQueue.add(streams.next());
+ }
final Map<String, Long> streamSpaceUsageMap =
new ConcurrentSkipListMap<String, Long>();
@@ -412,7 +422,7 @@
@Override
public void execute(String stream) throws IOException {
streamSpaceUsageMap.put(stream,
- calculateStreamSpaceUsage(factory, stream));
+ calculateStreamSpaceUsage(namespace, stream));
if (numStreamsCollected.incrementAndGet() % 1000 == 0) {
logger.info("Calculated {} streams from uri {}.", numStreamsCollected.get(), uri);
}
@@ -422,16 +432,15 @@
return streamSpaceUsageMap;
}
- private long calculateStreamSpaceUsage(final com.twitter.distributedlog.DistributedLogManagerFactory factory,
+ private long calculateStreamSpaceUsage(final DistributedLogNamespace namespace,
final String stream) throws IOException {
- DistributedLogManager dlm = factory.createDistributedLogManager(stream,
- com.twitter.distributedlog.DistributedLogManagerFactory.ClientSharingOption.SharedClients);
+ DistributedLogManager dlm = namespace.openLog(stream);
long totalBytes = 0;
try {
List<LogSegmentMetadata> segments = dlm.getLogSegments();
for (LogSegmentMetadata segment : segments) {
try {
- LedgerHandle lh = getBookKeeperClient(factory).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();
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKDLUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKDLUtils.java
deleted file mode 100644
index dd78a4e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKDLUtils.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * 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.
- */
-package com.twitter.distributedlog.impl;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-
-import java.net.URI;
-
-/**
- * Utils for bookkeeper based distributedlog implementation
- */
-public class BKDLUtils {
-
- /**
- * Is it a reserved stream name in bkdl namespace?
- *
- * @param name
- * stream name
- * @return true if it is reserved name, otherwise false.
- */
- public static boolean isReservedStreamName(String name) {
- return name.startsWith(".");
- }
-
- /**
- * Validate the configuration and uri.
- *
- * @param conf
- * distributedlog configuration
- * @param uri
- * distributedlog uri
- * @throws IllegalArgumentException
- */
- public static void validateConfAndURI(DistributedLogConfiguration conf, URI uri)
- throws IllegalArgumentException {
- if (null == conf) {
- throw new IllegalArgumentException("Incorrect Configuration");
- } else {
- conf.validate();
- }
- if ((null == uri) || (null == uri.getAuthority()) || (null == uri.getPath())) {
- throw new IllegalArgumentException("Incorrect ZK URI");
- }
- }
-
- /**
- * Validate the stream name.
- *
- * @param nameOfStream
- * name of stream
- * @throws InvalidStreamNameException
- */
- public static void validateName(String nameOfStream)
- throws InvalidStreamNameException {
- String reason = null;
- char chars[] = nameOfStream.toCharArray();
- char c;
- // validate the stream to see if meet zookeeper path's requirement
- for (int i = 0; i < chars.length; i++) {
- c = chars[i];
-
- if (c == 0) {
- reason = "null character not allowed @" + i;
- break;
- } else if (c == '/') {
- reason = "'/' not allowed @" + i;
- break;
- } else if (c > '\u0000' && c < '\u001f'
- || c > '\u007f' && c < '\u009F'
- || c > '\ud800' && c < '\uf8ff'
- || c > '\ufff0' && c < '\uffff') {
- reason = "invalid charater @" + i;
- break;
- }
- }
- if (null != reason) {
- throw new InvalidStreamNameException(nameOfStream, reason);
- }
- if (isReservedStreamName(nameOfStream)) {
- throw new InvalidStreamNameException(nameOfStream,
- "Stream Name is reserved");
- }
- }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java
new file mode 100644
index 0000000..5921233
--- /dev/null
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java
@@ -0,0 +1,631 @@
+/**
+ * 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.
+ */
+package com.twitter.distributedlog.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.twitter.distributedlog.BookKeeperClient;
+import com.twitter.distributedlog.BookKeeperClientBuilder;
+import com.twitter.distributedlog.DistributedLogConfiguration;
+import com.twitter.distributedlog.DistributedLogConstants;
+import com.twitter.distributedlog.MetadataAccessor;
+import com.twitter.distributedlog.ZooKeeperClient;
+import com.twitter.distributedlog.ZooKeeperClientBuilder;
+import com.twitter.distributedlog.acl.AccessControlManager;
+import com.twitter.distributedlog.acl.DefaultAccessControlManager;
+import com.twitter.distributedlog.impl.acl.ZKAccessControlManager;
+import com.twitter.distributedlog.bk.LedgerAllocator;
+import com.twitter.distributedlog.bk.LedgerAllocatorUtils;
+import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
+import com.twitter.distributedlog.exceptions.AlreadyClosedException;
+import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
+import com.twitter.distributedlog.impl.federated.FederatedZKLogMetadataStore;
+import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
+import com.twitter.distributedlog.impl.metadata.ZKLogStreamMetadataStore;
+import com.twitter.distributedlog.impl.subscription.ZKSubscriptionsStore;
+import com.twitter.distributedlog.injector.AsyncFailureInjector;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
+import com.twitter.distributedlog.metadata.LogMetadataForReader;
+import com.twitter.distributedlog.metadata.LogMetadataStore;
+import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
+import com.twitter.distributedlog.namespace.NamespaceDriverManager;
+import com.twitter.distributedlog.subscription.SubscriptionsStore;
+import com.twitter.distributedlog.util.OrderedScheduler;
+import com.twitter.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;
+import org.apache.zookeeper.data.Stat;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.jboss.netty.util.HashedWheelTimer;
+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.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static com.twitter.distributedlog.util.DLUtils.isReservedStreamName;
+import static com.twitter.distributedlog.util.DLUtils.validateName;
+
+/**
+ * Manager for ZooKeeper/BookKeeper based namespace
+ */
+public class BKNamespaceDriver implements NamespaceDriver {
+
+ private static Logger LOG = LoggerFactory.getLogger(BKNamespaceDriver.class);
+
+ // register itself
+ static {
+ NamespaceDriverManager.registerDriver(DistributedLogConstants.BACKEND_BK, BKNamespaceDriver.class);
+ }
+
+ /**
+ * Extract zk servers fro dl <i>namespace</i>.
+ *
+ * @param uri dl namespace
+ * @return zk servers
+ */
+ public static String getZKServersFromDLUri(URI uri) {
+ return uri.getAuthority().replace(";", ",");
+ }
+
+ // resources (passed from initialization)
+ private DistributedLogConfiguration conf;
+ private DynamicDistributedLogConfiguration dynConf;
+ private URI namespace;
+ private OrderedScheduler scheduler;
+ private FeatureProvider featureProvider;
+ private AsyncFailureInjector failureInjector;
+ private StatsLogger statsLogger;
+ private StatsLogger perLogStatsLogger;
+ private String clientId;
+ private int regionId;
+
+ //
+ // resources (created internally and initialized at #initialize())
+ //
+
+ // namespace binding
+ private BKDLConfig bkdlConfig;
+
+ // zookeeper clients
+ // NOTE: The actual zookeeper client is initialized lazily when it is referenced by
+ // {@link com.twitter.distributedlog.ZooKeeperClient#get()}. So it is safe to
+ // keep builders and their client wrappers here, as they will be used when
+ // instantiating readers or writers.
+ private ZooKeeperClientBuilder sharedWriterZKCBuilder;
+ private ZooKeeperClient writerZKC;
+ private ZooKeeperClientBuilder sharedReaderZKCBuilder;
+ private ZooKeeperClient readerZKC;
+ // NOTE: The actual bookkeeper client is initialized lazily when it is referenced by
+ // {@link com.twitter.distributedlog.BookKeeperClient#get()}. So it is safe to
+ // keep builders and their client wrappers here, as they will be used when
+ // instantiating readers or writers.
+ private ClientSocketChannelFactory channelFactory;
+ private HashedWheelTimer requestTimer;
+ private BookKeeperClientBuilder sharedWriterBKCBuilder;
+ private BookKeeperClient writerBKC;
+ private BookKeeperClientBuilder sharedReaderBKCBuilder;
+ private BookKeeperClient readerBKC;
+
+ // log stream metadata store
+ private LogMetadataStore metadataStore;
+ private LogStreamMetadataStore writerStreamMetadataStore;
+ private LogStreamMetadataStore readerStreamMetadataStore;
+
+ //
+ // resources (lazily initialized)
+ //
+
+ // ledger allocator
+ private LedgerAllocator allocator;
+
+ // log segment entry stores
+ private LogSegmentEntryStore writerEntryStore;
+ private LogSegmentEntryStore readerEntryStore;
+
+ // access control manager
+ private AccessControlManager accessControlManager;
+
+ //
+ // states
+ //
+ protected boolean initialized = false;
+ protected AtomicBoolean closed = new AtomicBoolean(false);
+
+ /**
+ * Public constructor for reflection.
+ */
+ public BKNamespaceDriver() {
+ }
+
+ @Override
+ public synchronized NamespaceDriver initialize(DistributedLogConfiguration conf,
+ DynamicDistributedLogConfiguration dynConf,
+ URI namespace,
+ OrderedScheduler scheduler,
+ FeatureProvider featureProvider,
+ AsyncFailureInjector failureInjector,
+ StatsLogger statsLogger,
+ StatsLogger perLogStatsLogger,
+ String clientId,
+ int regionId) throws IOException {
+ if (initialized) {
+ return this;
+ }
+ // validate the namespace
+ if ((null == namespace) || (null == namespace.getAuthority()) || (null == namespace.getPath())) {
+ throw new IOException("Incorrect distributedlog namespace : " + namespace);
+ }
+
+ // initialize the resources
+ this.conf = conf;
+ this.dynConf = dynConf;
+ this.namespace = namespace;
+ this.scheduler = scheduler;
+ this.featureProvider = featureProvider;
+ this.failureInjector = failureInjector;
+ this.statsLogger = statsLogger;
+ this.perLogStatsLogger = perLogStatsLogger;
+ this.clientId = clientId;
+ this.regionId = regionId;
+
+ // initialize the zookeeper clients
+ initializeZooKeeperClients();
+
+ // initialize the bookkeeper clients
+ initializeBookKeeperClients();
+
+ // propagate bkdlConfig to configuration
+ BKDLConfig.propagateConfiguration(bkdlConfig, conf);
+
+ // initialize the log metadata & stream metadata store
+ initializeLogStreamMetadataStores();
+
+ // initialize other resources
+ initializeOtherResources();
+
+ initialized = true;
+
+ LOG.info("Initialized BK namespace driver: clientId = {}, regionId = {}, federated = {}.",
+ new Object[]{clientId, regionId, bkdlConfig.isFederatedNamespace()});
+ return this;
+ }
+
+ private void initializeZooKeeperClients() throws IOException {
+ // Build the namespace zookeeper client
+ this.sharedWriterZKCBuilder = createZKClientBuilder(
+ String.format("dlzk:%s:factory_writer_shared", namespace),
+ conf,
+ getZKServersFromDLUri(namespace),
+ statsLogger.scope("dlzk_factory_writer_shared"));
+ this.writerZKC = sharedWriterZKCBuilder.build();
+
+ // Resolve namespace binding
+ this.bkdlConfig = BKDLConfig.resolveDLConfig(writerZKC, namespace);
+
+ // Build zookeeper client for readers
+ if (bkdlConfig.getDlZkServersForWriter().equals(bkdlConfig.getDlZkServersForReader())) {
+ this.sharedReaderZKCBuilder = this.sharedWriterZKCBuilder;
+ } else {
+ this.sharedReaderZKCBuilder = createZKClientBuilder(
+ String.format("dlzk:%s:factory_reader_shared", namespace),
+ conf,
+ bkdlConfig.getDlZkServersForReader(),
+ statsLogger.scope("dlzk_factory_reader_shared"));
+ }
+ this.readerZKC = this.sharedReaderZKCBuilder.build();
+ }
+
+ private synchronized BKDLConfig getBkdlConfig() {
+ return bkdlConfig;
+ }
+
+ private void initializeBookKeeperClients() throws IOException {
+ this.channelFactory = new NioClientSocketChannelFactory(
+ Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-boss-%d").build()),
+ Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-worker-%d").build()),
+ conf.getBKClientNumberIOThreads());
+ this.requestTimer = new HashedWheelTimer(
+ new ThreadFactoryBuilder().setNameFormat("DLFactoryTimer-%d").build(),
+ conf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS,
+ conf.getTimeoutTimerNumTicks());
+ // Build bookkeeper client for writers
+ this.sharedWriterBKCBuilder = createBKCBuilder(
+ String.format("bk:%s:factory_writer_shared", namespace),
+ conf,
+ bkdlConfig.getBkZkServersForWriter(),
+ bkdlConfig.getBkLedgersPath(),
+ channelFactory,
+ requestTimer,
+ Optional.of(featureProvider.scope("bkc")),
+ statsLogger);
+ this.writerBKC = this.sharedWriterBKCBuilder.build();
+
+ // Build bookkeeper client for readers
+ if (bkdlConfig.getBkZkServersForWriter().equals(bkdlConfig.getBkZkServersForReader())) {
+ this.sharedReaderBKCBuilder = this.sharedWriterBKCBuilder;
+ } else {
+ this.sharedReaderBKCBuilder = createBKCBuilder(
+ String.format("bk:%s:factory_reader_shared", namespace),
+ conf,
+ bkdlConfig.getBkZkServersForReader(),
+ bkdlConfig.getBkLedgersPath(),
+ channelFactory,
+ requestTimer,
+ Optional.<FeatureProvider>absent(),
+ statsLogger);
+ }
+ this.readerBKC = this.sharedReaderBKCBuilder.build();
+ }
+
+ private void initializeLogStreamMetadataStores() throws IOException {
+ // log metadata store
+ if (bkdlConfig.isFederatedNamespace() || conf.isFederatedNamespaceEnabled()) {
+ this.metadataStore = new FederatedZKLogMetadataStore(conf, namespace, readerZKC, scheduler);
+ } else {
+ this.metadataStore = new ZKLogMetadataStore(conf, namespace, readerZKC, scheduler);
+ }
+
+ // create log stream metadata store
+ this.writerStreamMetadataStore =
+ new ZKLogStreamMetadataStore(
+ clientId,
+ conf,
+ writerZKC,
+ scheduler,
+ statsLogger);
+ this.readerStreamMetadataStore =
+ new ZKLogStreamMetadataStore(
+ clientId,
+ conf,
+ readerZKC,
+ scheduler,
+ statsLogger);
+ }
+
+ @VisibleForTesting
+ 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);
+ throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
+ }
+ String poolName = conf.getLedgerAllocatorPoolName();
+ if (null == poolName) {
+ LOG.error("No ledger allocator pool name specified when enabling ledger allocator pool.");
+ throw new IOException("No ledger allocator name specified when enabling ledger allocator pool.");
+ }
+ String rootPath = uri.getPath() + "/" + poolPath + "/" + poolName;
+ try {
+ PathUtils.validatePath(rootPath);
+ } catch (IllegalArgumentException iae) {
+ 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;
+ }
+
+ private void initializeOtherResources() throws IOException {
+ // Ledger allocator
+ if (conf.getEnableLedgerAllocatorPool()) {
+ String allocatorPoolPath = validateAndGetFullLedgerAllocatorPoolPath(conf, namespace);
+ allocator = LedgerAllocatorUtils.createLedgerAllocatorPool(
+ allocatorPoolPath,
+ conf.getLedgerAllocatorPoolCoreSize(),
+ conf,
+ writerZKC,
+ writerBKC,
+ scheduler);
+ if (null != allocator) {
+ allocator.start();
+ }
+ LOG.info("Created ledger allocator pool under {} with size {}.", allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize());
+ } else {
+ allocator = null;
+ }
+
+ }
+
+ private void checkState() throws IOException {
+ if (closed.get()) {
+ LOG.error("BK namespace driver {} is already closed", namespace);
+ throw new AlreadyClosedException("BK namespace driver " + namespace + " is already closed");
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (!closed.compareAndSet(false, true)) {
+ return;
+ }
+ doClose();
+ }
+
+ private void doClose() {
+ if (null != accessControlManager) {
+ accessControlManager.close();
+ LOG.info("Access Control Manager Stopped.");
+ }
+
+ // Close the allocator
+ if (null != allocator) {
+ Utils.closeQuietly(allocator);
+ LOG.info("Ledger Allocator stopped.");
+ }
+
+ // Shutdown log segment metadata stores
+ Utils.close(writerStreamMetadataStore);
+ Utils.close(readerStreamMetadataStore);
+
+ writerBKC.close();
+ readerBKC.close();
+ writerZKC.close();
+ readerZKC.close();
+ // release bookkeeper resources
+ channelFactory.releaseExternalResources();
+ LOG.info("Release external resources used by channel factory.");
+ requestTimer.stop();
+ LOG.info("Stopped request timer");
+ }
+
+ @Override
+ public URI getUri() {
+ return namespace;
+ }
+
+ @Override
+ public String getScheme() {
+ return DistributedLogConstants.BACKEND_BK;
+ }
+
+ @Override
+ public LogMetadataStore getLogMetadataStore() {
+ return metadataStore;
+ }
+
+ @Override
+ public LogStreamMetadataStore getLogStreamMetadataStore(Role role) {
+ if (Role.WRITER == role) {
+ return writerStreamMetadataStore;
+ } else {
+ return readerStreamMetadataStore;
+ }
+ }
+
+ @Override
+ public LogSegmentEntryStore getLogSegmentEntryStore(Role role) {
+ if (Role.WRITER == role) {
+ return getWriterEntryStore();
+ } else {
+ return getReaderEntryStore();
+ }
+ }
+
+ private LogSegmentEntryStore getWriterEntryStore() {
+ if (null == writerEntryStore) {
+ writerEntryStore = new BKLogSegmentEntryStore(
+ conf,
+ dynConf,
+ writerZKC,
+ writerBKC,
+ scheduler,
+ allocator,
+ statsLogger,
+ failureInjector);
+ }
+ return writerEntryStore;
+ }
+
+ private LogSegmentEntryStore getReaderEntryStore() {
+ if (null == readerEntryStore) {
+ readerEntryStore = new BKLogSegmentEntryStore(
+ conf,
+ dynConf,
+ writerZKC,
+ readerBKC,
+ scheduler,
+ allocator,
+ statsLogger,
+ failureInjector);
+ }
+ return readerEntryStore;
+ }
+
+ @Override
+ public AccessControlManager getAccessControlManager() throws IOException {
+ if (null == accessControlManager) {
+ String aclRootPath = getBkdlConfig().getACLRootPath();
+ // Build the access control manager
+ if (aclRootPath == null) {
+ accessControlManager = DefaultAccessControlManager.INSTANCE;
+ LOG.info("Created default access control manager for {}", namespace);
+ } else {
+ if (!isReservedStreamName(aclRootPath)) {
+ throw new IOException("Invalid Access Control List Root Path : " + aclRootPath);
+ }
+ String zkRootPath = namespace.getPath() + "/" + aclRootPath;
+ LOG.info("Creating zk based access control manager @ {} for {}",
+ zkRootPath, namespace);
+ accessControlManager = new ZKAccessControlManager(conf, readerZKC,
+ zkRootPath, scheduler);
+ LOG.info("Created zk based access control manager @ {} for {}",
+ zkRootPath, namespace);
+ }
+ }
+ return accessControlManager;
+ }
+
+ @Override
+ public SubscriptionsStore getSubscriptionsStore(String streamName) {
+ return new ZKSubscriptionsStore(
+ writerZKC,
+ LogMetadataForReader.getSubscribersPath(namespace, streamName, conf.getUnpartitionedStreamName()));
+ }
+
+ //
+ // Legacy Intefaces
+ //
+
+ @Override
+ public MetadataAccessor getMetadataAccessor(String streamName)
+ throws InvalidStreamNameException, IOException {
+ if (getBkdlConfig().isFederatedNamespace()) {
+ throw new UnsupportedOperationException();
+ }
+ checkState();
+ validateName(streamName);
+ return new ZKMetadataAccessor(
+ streamName,
+ conf,
+ namespace,
+ sharedWriterZKCBuilder,
+ sharedReaderZKCBuilder,
+ statsLogger);
+ }
+
+ public Map<String, byte[]> enumerateLogsWithMetadataInNamespace()
+ throws IOException, IllegalArgumentException {
+ String namespaceRootPath = namespace.getPath();
+ HashMap<String, byte[]> result = new HashMap<String, byte[]>();
+ ZooKeeperClient zkc = writerZKC;
+ try {
+ ZooKeeper zk = Utils.sync(zkc, namespaceRootPath);
+ Stat currentStat = zk.exists(namespaceRootPath, false);
+ if (currentStat == null) {
+ return result;
+ }
+ List<String> children = zk.getChildren(namespaceRootPath, false);
+ for(String child: children) {
+ if (isReservedStreamName(child)) {
+ continue;
+ }
+ String zkPath = String.format("%s/%s", namespaceRootPath, child);
+ currentStat = zk.exists(zkPath, false);
+ if (currentStat == null) {
+ result.put(child, new byte[0]);
+ } else {
+ result.put(child, zk.getData(zkPath, false, currentStat));
+ }
+ }
+ } catch (InterruptedException ie) {
+ LOG.error("Interrupted while deleting " + namespaceRootPath, ie);
+ throw new IOException("Interrupted while reading " + namespaceRootPath, ie);
+ } catch (KeeperException ke) {
+ LOG.error("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
+ throw new IOException("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
+ }
+ return result;
+ }
+
+ //
+ // Zk & Bk Utils
+ //
+
+ public static ZooKeeperClientBuilder createZKClientBuilder(String zkcName,
+ DistributedLogConfiguration conf,
+ String zkServers,
+ StatsLogger statsLogger) {
+ RetryPolicy retryPolicy = null;
+ if (conf.getZKNumRetries() > 0) {
+ retryPolicy = new BoundExponentialBackoffRetryPolicy(
+ conf.getZKRetryBackoffStartMillis(),
+ conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
+ }
+ ZooKeeperClientBuilder builder = ZooKeeperClientBuilder.newBuilder()
+ .name(zkcName)
+ .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+ .retryThreadCount(conf.getZKClientNumberRetryThreads())
+ .requestRateLimit(conf.getZKRequestRateLimit())
+ .zkServers(zkServers)
+ .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() });
+ return builder;
+ }
+
+ private BookKeeperClientBuilder createBKCBuilder(String bkcName,
+ DistributedLogConfiguration conf,
+ String zkServers,
+ String ledgersPath,
+ ClientSocketChannelFactory channelFactory,
+ HashedWheelTimer requestTimer,
+ Optional<FeatureProvider> featureProviderOptional,
+ StatsLogger statsLogger) {
+ BookKeeperClientBuilder builder = BookKeeperClientBuilder.newBuilder()
+ .name(bkcName)
+ .dlConfig(conf)
+ .zkServers(zkServers)
+ .ledgersPath(ledgersPath)
+ .channelFactory(channelFactory)
+ .requestTimer(requestTimer)
+ .featureProvider(featureProviderOptional)
+ .statsLogger(statsLogger);
+ LOG.info("Created shared client builder {} : zkServers = {}, ledgersPath = {}, numIOThreads = {}",
+ new Object[] { bkcName, zkServers, ledgersPath, conf.getBKClientNumberIOThreads() });
+ return builder;
+ }
+
+ //
+ // Test Methods
+ //
+
+ @VisibleForTesting
+ public ZooKeeperClient getWriterZKC() {
+ return writerZKC;
+ }
+
+ @VisibleForTesting
+ public BookKeeperClient getReaderBKC() {
+ return readerBKC;
+ }
+
+ @VisibleForTesting
+ public AsyncFailureInjector getFailureInjector() {
+ return this.failureInjector;
+ }
+
+ @VisibleForTesting
+ public LogStreamMetadataStore getWriterStreamMetadataStore() {
+ return writerStreamMetadataStore;
+ }
+
+ @VisibleForTesting
+ public LedgerAllocator getLedgerAllocator() {
+ return allocator;
+ }
+}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
index b84ab2e..50b1405 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
@@ -36,7 +36,7 @@
import java.util.Iterator;
import java.util.List;
-import static com.twitter.distributedlog.impl.BKDLUtils.*;
+import static com.twitter.distributedlog.util.DLUtils.*;
/**
* ZooKeeper based log metadata store
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZKMetadataAccessor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
similarity index 95%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/ZKMetadataAccessor.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
index 4d7a0e1..eeda804 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZKMetadataAccessor.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
@@ -15,16 +15,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package com.twitter.distributedlog;
+package com.twitter.distributedlog.impl;
import java.io.IOException;
import java.net.URI;
import com.google.common.annotations.VisibleForTesting;
+import com.twitter.distributedlog.DistributedLogConfiguration;
+import com.twitter.distributedlog.MetadataAccessor;
+import com.twitter.distributedlog.ZooKeeperClient;
+import com.twitter.distributedlog.ZooKeeperClientBuilder;
import com.twitter.distributedlog.exceptions.AlreadyClosedException;
import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.metadata.BKDLConfig;
-import com.twitter.distributedlog.util.DLUtils;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.Utils;
import com.twitter.util.Future;
@@ -37,6 +40,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static com.twitter.distributedlog.impl.BKNamespaceDriver.getZKServersFromDLUri;
+
public class ZKMetadataAccessor implements MetadataAccessor {
static final Logger LOG = LoggerFactory.getLogger(ZKMetadataAccessor.class);
protected final String name;
@@ -87,7 +92,7 @@
this.writerZKC = this.writerZKCBuilder.build();
if (null == readerZKCBuilder) {
- String zkServersForWriter = DLUtils.getZKServersFromDLUri(uri);
+ String zkServersForWriter = getZKServersFromDLUri(uri);
String zkServersForReader;
try {
BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(this.writerZKC, uri);
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
index 61c1760..06bc8fb 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
@@ -37,7 +37,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
-import static com.twitter.distributedlog.impl.BKDLUtils.*;
+import static com.twitter.distributedlog.util.DLUtils.*;
/**
* Watcher on watching a given namespace
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControl.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
similarity index 98%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControl.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
index bf16256..8126723 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControl.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
@@ -15,8 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package com.twitter.distributedlog.acl;
+package com.twitter.distributedlog.impl.acl;
+import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.thrift.AccessControlEntry;
@@ -89,10 +90,12 @@
return sb.toString();
}
+ @VisibleForTesting
public String getZKPath() {
return zkPath;
}
+ @VisibleForTesting
public AccessControlEntry getAccessControlEntry() {
return accessControlEntry;
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControlManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
similarity index 99%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControlManager.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
index 9c89b4a..0c90a50 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/ZKAccessControlManager.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
@@ -15,11 +15,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package com.twitter.distributedlog.acl;
+package com.twitter.distributedlog.impl.acl;
import com.google.common.collect.Sets;
import com.twitter.distributedlog.DistributedLogConfiguration;
import com.twitter.distributedlog.ZooKeeperClient;
+import com.twitter.distributedlog.acl.AccessControlManager;
import com.twitter.distributedlog.exceptions.DLInterruptedException;
import com.twitter.distributedlog.thrift.AccessControlEntry;
import com.twitter.util.Await;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
new file mode 100644
index 0000000..d7ff4fb
--- /dev/null
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
@@ -0,0 +1,85 @@
+/**
+ * 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.
+ */
+package com.twitter.distributedlog.impl.logsegment;
+
+import com.twitter.distributedlog.bk.LedgerAllocator;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
+import com.twitter.distributedlog.util.Allocator;
+import com.twitter.distributedlog.util.Transaction;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.client.LedgerHandle;
+import scala.Function1;
+import scala.runtime.AbstractFunction1;
+
+import java.io.IOException;
+
+/**
+ * Allocate log segments
+ */
+class BKLogSegmentAllocator implements Allocator<LogSegmentEntryWriter, Object> {
+
+ private static class NewLogSegmentEntryWriterFn extends AbstractFunction1<LedgerHandle, LogSegmentEntryWriter> {
+
+ static final Function1<LedgerHandle, LogSegmentEntryWriter> INSTANCE =
+ new NewLogSegmentEntryWriterFn();
+
+ private NewLogSegmentEntryWriterFn() {}
+
+ @Override
+ public LogSegmentEntryWriter apply(LedgerHandle lh) {
+ return new BKLogSegmentEntryWriter(lh);
+ }
+ }
+
+ LedgerAllocator allocator;
+
+ BKLogSegmentAllocator(LedgerAllocator allocator) {
+ this.allocator = allocator;
+ }
+
+ @Override
+ public void allocate() throws IOException {
+ allocator.allocate();
+ }
+
+ @Override
+ public Future<LogSegmentEntryWriter> tryObtain(Transaction<Object> txn,
+ final Transaction.OpListener<LogSegmentEntryWriter> listener) {
+ return allocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
+ @Override
+ public void onCommit(LedgerHandle lh) {
+ listener.onCommit(new BKLogSegmentEntryWriter(lh));
+ }
+
+ @Override
+ public void onAbort(Throwable t) {
+ listener.onAbort(t);
+ }
+ }).map(NewLogSegmentEntryWriterFn.INSTANCE);
+ }
+
+ @Override
+ public Future<Void> asyncClose() {
+ return allocator.asyncClose();
+ }
+
+ @Override
+ public Future<Void> delete() {
+ return allocator.delete();
+ }
+}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
index dc382d2..f85760d 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
@@ -48,6 +48,7 @@
import java.util.Enumeration;
import java.util.LinkedList;
import java.util.List;
+import java.util.concurrent.CopyOnWriteArraySet;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
@@ -209,6 +210,7 @@
int numErrors = Math.max(1, numReadErrors.incrementAndGet());
int nextReadBackoffTime = Math.min(numErrors * readAheadWaitTime, maxReadBackoffTime);
scheduler.schedule(
+ getSegment().getLogSegmentId(),
this,
nextReadBackoffTime,
TimeUnit.MILLISECONDS);
@@ -284,6 +286,8 @@
private final AtomicReference<Throwable> lastException = new AtomicReference<Throwable>(null);
private final AtomicLong scheduleCount = new AtomicLong(0);
private volatile boolean hasCaughtupOnInprogress = false;
+ private final CopyOnWriteArraySet<StateChangeListener> stateChangeListeners =
+ new CopyOnWriteArraySet<StateChangeListener>();
// read retries
private int readAheadWaitTime;
private final int maxReadBackoffTime;
@@ -374,6 +378,24 @@
return hasCaughtupOnInprogress;
}
+ @Override
+ public LogSegmentEntryReader registerListener(StateChangeListener listener) {
+ stateChangeListeners.add(listener);
+ return this;
+ }
+
+ @Override
+ public LogSegmentEntryReader unregisterListener(StateChangeListener listener) {
+ stateChangeListeners.remove(listener);
+ return this;
+ }
+
+ private void notifyCaughtupOnInprogress() {
+ for (StateChangeListener listener : stateChangeListeners) {
+ listener.onCaughtupOnInprogress();
+ }
+ }
+
//
// Process on Log Segment Metadata Updates
//
@@ -440,7 +462,7 @@
return;
}
// the reader is still catching up, retry opening the log segment later
- scheduler.schedule(new Runnable() {
+ scheduler.schedule(segment.getLogSegmentId(), new Runnable() {
@Override
public void run() {
onLogSegmentMetadataUpdated(segment);
@@ -583,6 +605,7 @@
if (!hasCaughtupOnInprogress) {
hasCaughtupOnInprogress = true;
+ notifyCaughtupOnInprogress();
}
getLh().asyncReadLastConfirmedAndEntry(
cacheEntry.entryId,
@@ -633,7 +656,7 @@
long prevCount = scheduleCount.getAndIncrement();
if (0 == prevCount) {
- scheduler.submit(this);
+ scheduler.submit(getSegment().getLogSegmentId(), this);
}
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
index f7f4acf..91e6dec 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
@@ -20,12 +20,21 @@
import com.twitter.distributedlog.BookKeeperClient;
import com.twitter.distributedlog.DistributedLogConfiguration;
import com.twitter.distributedlog.LogSegmentMetadata;
+import com.twitter.distributedlog.ZooKeeperClient;
+import com.twitter.distributedlog.bk.DynamicQuorumConfigProvider;
+import com.twitter.distributedlog.bk.LedgerAllocator;
+import com.twitter.distributedlog.bk.LedgerAllocatorDelegator;
+import com.twitter.distributedlog.bk.QuorumConfigProvider;
+import com.twitter.distributedlog.bk.SimpleLedgerAllocator;
+import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
import com.twitter.distributedlog.exceptions.BKTransmitException;
import com.twitter.distributedlog.injector.AsyncFailureInjector;
import com.twitter.distributedlog.logsegment.LogSegmentEntryReader;
import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
import com.twitter.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
+import com.twitter.distributedlog.metadata.LogMetadataForWriter;
+import com.twitter.distributedlog.util.Allocator;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.OrderedScheduler;
import com.twitter.util.Future;
@@ -80,21 +89,31 @@
}
private final byte[] passwd;
+ private final ZooKeeperClient zkc;
private final BookKeeperClient bkc;
private final OrderedScheduler scheduler;
private final DistributedLogConfiguration conf;
+ private final DynamicDistributedLogConfiguration dynConf;
private final StatsLogger statsLogger;
private final AsyncFailureInjector failureInjector;
+ // ledger allocator
+ private final LedgerAllocator allocator;
public BKLogSegmentEntryStore(DistributedLogConfiguration conf,
+ DynamicDistributedLogConfiguration dynConf,
+ ZooKeeperClient zkc,
BookKeeperClient bkc,
OrderedScheduler scheduler,
+ LedgerAllocator allocator,
StatsLogger statsLogger,
AsyncFailureInjector failureInjector) {
this.conf = conf;
+ this.dynConf = dynConf;
+ this.zkc = zkc;
this.bkc = bkc;
this.passwd = conf.getBKDigestPW().getBytes(UTF_8);
this.scheduler = scheduler;
+ this.allocator = allocator;
this.statsLogger = statsLogger;
this.failureInjector = failureInjector;
}
@@ -129,12 +148,44 @@
FutureUtils.setValue(deleteRequest.deletePromise, deleteRequest.segment);
}
- @Override
- public Future<LogSegmentEntryWriter> openWriter(LogSegmentMetadata segment) {
- throw new UnsupportedOperationException("Not supported yet");
+ //
+ // Writers
+ //
+
+ LedgerAllocator createLedgerAllocator(LogMetadataForWriter logMetadata,
+ DynamicDistributedLogConfiguration dynConf)
+ throws IOException {
+ LedgerAllocator ledgerAllocatorDelegator;
+ if (null == allocator || !dynConf.getEnableLedgerAllocatorPool()) {
+ QuorumConfigProvider quorumConfigProvider =
+ new DynamicQuorumConfigProvider(dynConf);
+ LedgerAllocator allocator = new SimpleLedgerAllocator(
+ logMetadata.getAllocationPath(),
+ logMetadata.getAllocationData(),
+ quorumConfigProvider,
+ zkc,
+ bkc);
+ ledgerAllocatorDelegator = new LedgerAllocatorDelegator(allocator, true);
+ } else {
+ ledgerAllocatorDelegator = allocator;
+ }
+ return ledgerAllocatorDelegator;
}
@Override
+ public Allocator<LogSegmentEntryWriter, Object> newLogSegmentAllocator(
+ LogMetadataForWriter logMetadata,
+ DynamicDistributedLogConfiguration dynConf) throws IOException {
+ // Build the ledger allocator
+ LedgerAllocator allocator = createLedgerAllocator(logMetadata, dynConf);
+ return new BKLogSegmentAllocator(allocator);
+ }
+
+ //
+ // Readers
+ //
+
+ @Override
public Future<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
long startEntryId) {
BookKeeper bk;
@@ -220,15 +271,15 @@
segment.getLogSegmentId(),
BookKeeper.DigestType.CRC32,
passwd,
- this,
- openCallback);
+ openCallback,
+ null);
} else {
bk.asyncOpenLedger(
segment.getLogSegmentId(),
BookKeeper.DigestType.CRC32,
passwd,
- this,
- openCallback);
+ openCallback,
+ null);
}
return openPromise;
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/BKDLConfig.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
similarity index 97%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/BKDLConfig.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
index ac36ef2..3e859fb 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/BKDLConfig.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
@@ -15,15 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package com.twitter.distributedlog.metadata;
+package com.twitter.distributedlog.impl.metadata;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Objects;
import com.twitter.distributedlog.DistributedLogConfiguration;
import com.twitter.distributedlog.DistributedLogConstants;
import com.twitter.distributedlog.ZooKeeperClient;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
+import com.twitter.distributedlog.metadata.DLConfig;
import com.twitter.distributedlog.thrift.BKDLConfigFormat;
-import com.twitter.distributedlog.util.DLUtils;
import org.apache.thrift.TException;
import org.apache.thrift.protocol.TJSONProtocol;
import org.apache.thrift.transport.TMemoryBuffer;
@@ -96,9 +97,9 @@
/**
* Construct a empty config with given <i>uri</i>.
*/
- BKDLConfig(URI uri) {
- this(DLUtils.getZKServersFromDLUri(uri),
- DLUtils.getZKServersFromDLUri(uri),
+ public BKDLConfig(URI uri) {
+ this(BKNamespaceDriver.getZKServersFromDLUri(uri),
+ BKNamespaceDriver.getZKServersFromDLUri(uri),
null, null, null);
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/ZkMetadataResolver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
similarity index 94%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/ZkMetadataResolver.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
index 303fbe6..6b7a231 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/ZkMetadataResolver.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
@@ -15,9 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package com.twitter.distributedlog.metadata;
+package com.twitter.distributedlog.impl.metadata;
import com.twitter.distributedlog.ZooKeeperClient;
+import com.twitter.distributedlog.metadata.DLMetadata;
+import com.twitter.distributedlog.metadata.MetadataResolver;
import org.apache.commons.lang.StringUtils;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.common.PathUtils;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionStateStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
similarity index 97%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionStateStore.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
index 9cd2da5..b067ee9 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionStateStore.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
@@ -15,11 +15,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package com.twitter.distributedlog.subscription;
+package com.twitter.distributedlog.impl.subscription;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicReference;
+import com.twitter.distributedlog.subscription.SubscriptionStateStore;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.runtime.BoxedUnit;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionsStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
similarity index 96%
rename from distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionsStore.java
rename to distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
index f1e6251..17ba943 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/ZKSubscriptionsStore.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
@@ -15,13 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package com.twitter.distributedlog.subscription;
+package com.twitter.distributedlog.impl.subscription;
import com.twitter.distributedlog.DLSN;
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.exceptions.DLInterruptedException;
+import com.twitter.distributedlog.subscription.SubscriptionStateStore;
+import com.twitter.distributedlog.subscription.SubscriptionsStore;
import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Function;
import com.twitter.util.Future;
import com.twitter.util.Promise;
@@ -30,8 +31,6 @@
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 scala.runtime.AbstractFunction1;
import scala.runtime.BoxedUnit;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
index 07387cb..81eb5ed 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
@@ -32,6 +32,15 @@
@Beta
public interface LogSegmentEntryReader extends AsyncCloseable {
+ interface StateChangeListener {
+
+ /**
+ * Notify when caught up on inprogress.
+ */
+ void onCaughtupOnInprogress();
+
+ }
+
/**
* Start the reader. The method to signal the implementation
* to start preparing the data for consumption {@link #readNext(int)}
@@ -39,6 +48,22 @@
void start();
/**
+ * Register the state change listener
+ *
+ * @param listener register the state change listener
+ * @return entry reader
+ */
+ LogSegmentEntryReader registerListener(StateChangeListener listener);
+
+ /**
+ * Unregister the state change listener
+ *
+ * @param listener register the state change listener
+ * @return entry reader
+ */
+ LogSegmentEntryReader unregisterListener(StateChangeListener listener);
+
+ /**
* Return the log segment metadata for this reader.
*
* @return the log segment metadata
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
index 850f9c8..bcf8129 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
@@ -19,8 +19,14 @@
import com.google.common.annotations.Beta;
import com.twitter.distributedlog.LogSegmentMetadata;
+import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
+import com.twitter.distributedlog.metadata.LogMetadataForWriter;
+import com.twitter.distributedlog.util.Allocator;
+import com.twitter.distributedlog.util.Transaction;
import com.twitter.util.Future;
+import java.io.IOException;
+
/**
* Log Segment Store to read log segments
*/
@@ -36,12 +42,14 @@
Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment);
/**
- * Open the writer for writing data to the log <i>segment</i>.
+ * Create a new log segment allocator for allocating log segment entry writers.
*
- * @param segment the log <i>segment</i> to write data to
- * @return future represent the opened writer
+ * @param metadata the metadata for the log stream
+ * @return future represent the log segment allocator
*/
- Future<LogSegmentEntryWriter> openWriter(LogSegmentMetadata segment);
+ Allocator<LogSegmentEntryWriter, Object> newLogSegmentAllocator(
+ LogMetadataForWriter metadata,
+ DynamicDistributedLogConfiguration dynConf) throws IOException;
/**
* Open the reader for reading data to the log <i>segment</i>.
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
index e0331c6..c0b5fb7 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
@@ -18,6 +18,7 @@
package com.twitter.distributedlog.metadata;
import com.twitter.distributedlog.DistributedLogConfiguration;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import com.twitter.distributedlog.util.Utils;
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.ZooKeeperClientBuilder;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
index b5abe9f..5d1d888 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
@@ -70,6 +70,13 @@
@Beta
public interface DistributedLogNamespace {
+ /**
+ * Get the namespace driver used by this namespace.
+ *
+ * @return namespace driver
+ */
+ NamespaceDriver getNamespaceDriver();
+
//
// Method to operate logs
//
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
index a01bb70..07b3848 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
@@ -17,22 +17,29 @@
*/
package com.twitter.distributedlog.namespace;
-import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.twitter.distributedlog.BKDistributedLogNamespace;
import com.twitter.distributedlog.DistributedLogConfiguration;
import com.twitter.distributedlog.DistributedLogConstants;
+import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
+import com.twitter.distributedlog.feature.CoreFeatureKeys;
+import com.twitter.distributedlog.injector.AsyncFailureInjector;
+import com.twitter.distributedlog.injector.AsyncRandomFailureInjector;
+import com.twitter.distributedlog.util.ConfUtils;
+import com.twitter.distributedlog.util.DLUtils;
+import com.twitter.distributedlog.util.OrderedScheduler;
+import com.twitter.distributedlog.util.PermitLimiter;
+import com.twitter.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.apache.commons.lang.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URI;
-import java.net.URISyntaxException;
/**
* Builder to construct a <code>DistributedLogNamespace</code>.
@@ -50,6 +57,7 @@
}
private DistributedLogConfiguration _conf = null;
+ private DynamicDistributedLogConfiguration _dynConf = null;
private URI _uri = null;
private StatsLogger _statsLogger = NullStatsLogger.INSTANCE;
private StatsLogger _perLogStatsLogger = NullStatsLogger.INSTANCE;
@@ -73,6 +81,17 @@
}
/**
+ * Dynamic DistributedLog Configuration used for the namespace
+ *
+ * @param dynConf dynamic distributedlog configuration
+ * @return namespace builder
+ */
+ public DistributedLogNamespaceBuilder dynConf(DynamicDistributedLogConfiguration dynConf) {
+ this._dynConf = dynConf;
+ return this;
+ }
+
+ /**
* Namespace Location.
*
* @param uri
@@ -146,6 +165,18 @@
return this;
}
+ @SuppressWarnings("deprecation")
+ private static StatsLogger normalizePerLogStatsLogger(StatsLogger statsLogger,
+ StatsLogger perLogStatsLogger,
+ DistributedLogConfiguration conf) {
+ StatsLogger normalizedPerLogStatsLogger = perLogStatsLogger;
+ if (perLogStatsLogger == NullStatsLogger.INSTANCE &&
+ conf.getEnablePerStreamStat()) {
+ normalizedPerLogStatsLogger = statsLogger.scope("stream");
+ }
+ return normalizedPerLogStatsLogger;
+ }
+
/**
* Build the namespace.
*
@@ -160,25 +191,17 @@
Preconditions.checkNotNull(_conf, "No DistributedLog Configuration.");
Preconditions.checkNotNull(_uri, "No DistributedLog URI");
- // Validate the uri and load the backend according to scheme
- String scheme = _uri.getScheme();
- Preconditions.checkNotNull(scheme, "Invalid DistributedLog URI : " + _uri);
- String[] schemeParts = StringUtils.split(scheme, '-');
- Preconditions.checkArgument(schemeParts.length > 0,
- "Invalid distributedlog scheme found : " + _uri);
- Preconditions.checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
- "Unknown distributedlog scheme found : " + _uri);
-
- // both distributedlog: & distributedlog-bk: use bookkeeper as the backend
- // TODO: we could do reflection to load backend in future.
- // if we are going to support other backends : e.g. 'distributedlog-mem:'.
- if (schemeParts.length > 1) {
- String backendProvider = schemeParts[1];
- Preconditions.checkArgument(Objects.equal(DistributedLogConstants.BACKEND_BK, backendProvider.toLowerCase()),
- "Backend '" + backendProvider + "' is not supported yet.");
+ // validate the configuration
+ _conf.validate();
+ if (null == _dynConf) {
+ _dynConf = ConfUtils.getConstDynConf(_conf);
}
- // Built the feature provider
+ // retrieve the namespace driver
+ NamespaceDriver driver = NamespaceDriverManager.getDriver(_uri);
+ URI normalizedUri = DLUtils.normalizeURI(_uri);
+
+ // build the feature provider
FeatureProvider featureProvider;
if (null == _featureProvider) {
featureProvider = new SettableFeatureProvider("", 0);
@@ -187,25 +210,69 @@
featureProvider = _featureProvider;
}
- URI bkUri;
- try {
- bkUri = new URI(
- schemeParts[0], // remove backend info from bookkeeper backend
- _uri.getAuthority(),
- _uri.getPath(),
- _uri.getQuery(),
- _uri.getFragment());
- } catch (URISyntaxException e) {
- throw new IllegalArgumentException("Invalid distributedlog uri found : " + _uri, e);
+ // build the failure injector
+ AsyncFailureInjector failureInjector = AsyncRandomFailureInjector.newBuilder()
+ .injectDelays(_conf.getEIInjectReadAheadDelay(),
+ _conf.getEIInjectReadAheadDelayPercent(),
+ _conf.getEIInjectMaxReadAheadDelayMs())
+ .injectErrors(false, 10)
+ .injectStops(_conf.getEIInjectReadAheadStall(), 10)
+ .injectCorruption(_conf.getEIInjectReadAheadBrokenEntries())
+ .build();
+
+ // normalize the per log stats logger
+ StatsLogger perLogStatsLogger = normalizePerLogStatsLogger(_statsLogger, _perLogStatsLogger, _conf);
+
+ // build the scheduler
+ StatsLogger schedulerStatsLogger = _statsLogger.scope("factory").scope("thread_pool");
+ OrderedScheduler scheduler = OrderedScheduler.newBuilder()
+ .name("DLM-" + normalizedUri.getPath())
+ .corePoolSize(_conf.getNumWorkerThreads())
+ .statsLogger(schedulerStatsLogger)
+ .perExecutorStatsLogger(schedulerStatsLogger)
+ .traceTaskExecution(_conf.getEnableTaskExecutionStats())
+ .traceTaskExecutionWarnTimeUs(_conf.getTaskExecutionWarnTimeMicros())
+ .build();
+
+ // initialize the namespace driver
+ driver.initialize(
+ _conf,
+ _dynConf,
+ normalizedUri,
+ scheduler,
+ featureProvider,
+ failureInjector,
+ _statsLogger,
+ perLogStatsLogger,
+ DLUtils.normalizeClientId(_clientId),
+ _regionId);
+
+ // initialize the write limiter
+ PermitLimiter writeLimiter;
+ if (_conf.getGlobalOutstandingWriteLimit() < 0) {
+ writeLimiter = PermitLimiter.NULL_PERMIT_LIMITER;
+ } else {
+ Feature disableWriteLimitFeature = featureProvider.getFeature(
+ CoreFeatureKeys.DISABLE_WRITE_LIMIT.name().toLowerCase());
+ writeLimiter = new SimplePermitLimiter(
+ _conf.getOutstandingWriteLimitDarkmode(),
+ _conf.getGlobalOutstandingWriteLimit(),
+ _statsLogger.scope("writeLimiter"),
+ true /* singleton */,
+ disableWriteLimitFeature);
}
- return BKDistributedLogNamespace.newBuilder()
- .conf(_conf)
- .uri(bkUri)
- .statsLogger(_statsLogger)
- .featureProvider(featureProvider)
- .clientId(_clientId)
- .regionId(_regionId)
- .build();
+ return new BKDistributedLogNamespace(
+ _conf,
+ normalizedUri,
+ driver,
+ scheduler,
+ featureProvider,
+ writeLimiter,
+ failureInjector,
+ _statsLogger,
+ perLogStatsLogger,
+ DLUtils.normalizeClientId(_clientId),
+ _regionId);
}
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java
new file mode 100644
index 0000000..738f124
--- /dev/null
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java
@@ -0,0 +1,138 @@
+/**
+ * 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.
+ */
+package com.twitter.distributedlog.namespace;
+
+import com.twitter.distributedlog.DistributedLogConfiguration;
+import com.twitter.distributedlog.MetadataAccessor;
+import com.twitter.distributedlog.acl.AccessControlManager;
+import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
+import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
+import com.twitter.distributedlog.injector.AsyncFailureInjector;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
+import com.twitter.distributedlog.metadata.LogMetadataStore;
+import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
+import com.twitter.distributedlog.subscription.SubscriptionsStore;
+import com.twitter.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 {
+
+ enum Role {
+ WRITER,
+ READER
+ }
+
+ /**
+ * Initialize the namespace manager.
+ *
+ * @param conf distributedlog configuration
+ * @param dynConf dynamic distributedlog configuration
+ * @param namespace root uri of the namespace
+ * @param scheduler ordered scheduler
+ * @param featureProvider feature provider
+ * @param statsLogger stats logger
+ * @param perLogStatsLogger per log stream stats logger
+ * @param clientId client id
+ * @return namespace manager
+ * @throws IOException when failed to initialize the namespace manager
+ */
+ NamespaceDriver initialize(DistributedLogConfiguration conf,
+ DynamicDistributedLogConfiguration dynConf,
+ URI namespace,
+ OrderedScheduler scheduler,
+ FeatureProvider featureProvider,
+ AsyncFailureInjector failureInjector,
+ StatsLogger statsLogger,
+ StatsLogger perLogStatsLogger,
+ String clientId,
+ int regionId) throws IOException;
+
+ /**
+ * Get the scheme of the namespace driver.
+ *
+ * @return the scheme of the namespace driver.
+ */
+ String getScheme();
+
+ /**
+ * Get the root uri of the namespace driver.
+ *
+ * @return the root uri of the namespace driver.
+ */
+ URI getUri();
+
+ /**
+ * Retrieve the log {@code metadata store} used by the namespace.
+ *
+ * @return the log metadata store
+ */
+ LogMetadataStore getLogMetadataStore();
+
+ /**
+ * Retrieve the log stream {@code metadata store} used by the namespace.
+ *
+ * @param role the role to retrieve the log stream metadata store.
+ * @return the log stream metadata store
+ */
+ LogStreamMetadataStore getLogStreamMetadataStore(Role role);
+
+ /**
+ * Retrieve the log segment {@code entry store} used by the namespace.
+ *
+ * @param role the role to retrieve the log segment entry store.
+ * @return the log segment entry store.
+ * @throws IOException when failed to open log segment entry store.
+ */
+ LogSegmentEntryStore getLogSegmentEntryStore(Role role);
+
+ /**
+ * Create an access control manager to manage/check acl for logs.
+ *
+ * @return access control manager for logs under the namespace.
+ * @throws IOException
+ */
+ AccessControlManager getAccessControlManager()
+ throws IOException;
+
+ /**
+ * Retrieve the metadata accessor for log stream {@code streamName}.
+ * (TODO: it is a legacy interface. should remove it if we have metadata of stream.)
+ *
+ * @param streamName name of log stream.
+ * @return metadata accessor for log stream {@code streamName}.
+ */
+ MetadataAccessor getMetadataAccessor(String streamName)
+ throws InvalidStreamNameException, IOException;
+
+ /**
+ * Retrieve the subscriptions store for log stream {@code streamName}.
+ *
+ * @return the subscriptions store for log stream {@code streamName}
+ */
+ SubscriptionsStore getSubscriptionsStore(String streamName);
+
+}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java
new file mode 100644
index 0000000..79945ad
--- /dev/null
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java
@@ -0,0 +1,180 @@
+/**
+ * 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.
+ */
+package com.twitter.distributedlog.namespace;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Sets;
+import com.twitter.distributedlog.DistributedLogConstants;
+import com.twitter.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 static com.google.common.base.Preconditions.*;
+
+/**
+ * The basic service for managing a set of namespace drivers.
+ */
+public class NamespaceDriverManager {
+
+ private static final Logger logger = LoggerFactory.getLogger(NamespaceDriverManager.class);
+
+ static class NamespaceDriverInfo {
+
+ final Class<? extends NamespaceDriver> driverClass;
+ final String driverClassName;
+
+ NamespaceDriverInfo(Class<? extends NamespaceDriver> driverClass) {
+ this.driverClass = driverClass;
+ this.driverClassName = this.driverClass.getName();
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("driver[")
+ .append(driverClassName)
+ .append("]");
+ return sb.toString();
+ }
+ }
+
+ private static final ConcurrentMap<String, NamespaceDriverInfo> drivers;
+ private static boolean initialized = false;
+
+ static {
+ drivers = new ConcurrentHashMap<String, NamespaceDriverInfo>();
+ initialize();
+ }
+
+ static void initialize() {
+ if (initialized) {
+ return;
+ }
+ loadInitialDrivers();
+ initialized = true;
+ logger.info("DistributedLog NamespaceDriverManager initialized");
+ }
+
+ private static void loadInitialDrivers() {
+ Set<String> driverList = Sets.newHashSet();
+ // add default bookkeeper based driver
+ driverList.add(BKNamespaceDriver.class.getName());
+ // load drivers from system property
+ String driversStr = System.getProperty("distributedlog.namespace.drivers");
+ if (null != driversStr) {
+ String[] driversArray = StringUtils.split(driversStr, ':');
+ for (String driver : driversArray) {
+ driverList.add(driver);
+ }
+ }
+ // initialize the drivers
+ for (String driverClsName : driverList) {
+ try {
+ NamespaceDriver driver =
+ ReflectionUtils.newInstance(driverClsName, NamespaceDriver.class);
+ NamespaceDriverInfo driverInfo = new NamespaceDriverInfo(driver.getClass());
+ drivers.put(driver.getScheme().toLowerCase(), driverInfo);
+ } catch (Exception ex) {
+ logger.warn("Failed to load namespace driver {} : ", driverClsName, ex);
+ }
+ }
+ }
+
+ /**
+ * Prevent the NamespaceDriverManager class from being instantiated.
+ */
+ private NamespaceDriverManager() {}
+
+ /**
+ * 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) {
+ initialize();
+ }
+
+ String scheme = backend.toLowerCase();
+ NamespaceDriverInfo oldDriverInfo = drivers.get(scheme);
+ if (null != oldDriverInfo) {
+ return;
+ }
+ NamespaceDriverInfo newDriverInfo = new NamespaceDriverInfo(driver);
+ oldDriverInfo = drivers.putIfAbsent(scheme, newDriverInfo);
+ if (null != oldDriverInfo) {
+ logger.debug("Driver for {} is already there.", scheme);
+ }
+ }
+
+ /**
+ * Retrieve the namespace driver for {@code scheme}.
+ *
+ * @param scheme the scheme for the namespace driver
+ * @return the namespace driver
+ * @throws NullPointerException when scheme is null
+ */
+ public static NamespaceDriver getDriver(String scheme) {
+ checkNotNull(scheme, "Driver Scheme is null");
+ if (!initialized) {
+ initialize();
+ }
+ NamespaceDriverInfo driverInfo = drivers.get(scheme.toLowerCase());
+ if (null == driverInfo) {
+ throw new IllegalArgumentException("Unknown backend " + scheme);
+ }
+ return ReflectionUtils.newInstance(driverInfo.driverClass);
+ }
+
+ /**
+ * Retrieve the namespace driver for {@code uri}.
+ *
+ * @param uri the distributedlog uri
+ * @return the namespace driver for {@code uri}
+ * @throws NullPointerException if the distributedlog {@code uri} is null or doesn't have scheme
+ * or there is no namespace driver registered for the scheme
+ * @throws IllegalArgumentException if the distributedlog {@code uri} scheme is illegal
+ */
+ public static NamespaceDriver getDriver(URI uri) {
+ // Validate the uri and load the backend according to scheme
+ checkNotNull(uri, "DistributedLog uri is null");
+ String scheme = uri.getScheme();
+ checkNotNull(scheme, "Invalid distributedlog uri : " + uri);
+ scheme = scheme.toLowerCase();
+ String[] schemeParts = StringUtils.split(scheme, '-');
+ checkArgument(schemeParts.length > 0,
+ "Invalid distributedlog scheme found : " + uri);
+ checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
+ "Unknown distributedlog scheme found : " + uri);
+ // bookkeeper is the default backend
+ String backend = DistributedLogConstants.BACKEND_BK;
+ if (schemeParts.length > 1) {
+ backend = schemeParts[1];
+ }
+ return getDriver(backend);
+ }
+
+}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
index 30d6908..4565921 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
@@ -30,7 +30,6 @@
import java.net.MalformedURLException;
import java.net.URI;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.Enumeration;
@@ -53,11 +52,16 @@
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import com.google.common.base.Preconditions;
import com.twitter.distributedlog.BKDistributedLogNamespace;
import com.twitter.distributedlog.Entry;
+import com.twitter.distributedlog.MetadataAccessor;
import com.twitter.distributedlog.callback.NamespaceListener;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
import com.twitter.distributedlog.util.Utils;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
@@ -100,17 +104,15 @@
import com.twitter.distributedlog.auditor.DLAuditor;
import com.twitter.distributedlog.bk.LedgerAllocator;
import com.twitter.distributedlog.bk.LedgerAllocatorUtils;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import com.twitter.distributedlog.metadata.MetadataUpdater;
import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
import com.twitter.distributedlog.util.SchedulerUtils;
import com.twitter.util.Await;
-import com.twitter.util.Future;
import com.twitter.util.FutureEventListener;
import static com.google.common.base.Charsets.UTF_8;
-@SuppressWarnings("deprecation")
public class DistributedLogTool extends Tool {
static final Logger logger = LoggerFactory.getLogger(DistributedLogTool.class);
@@ -161,7 +163,7 @@
protected URI uri;
protected String zkAclId = null;
protected boolean force = false;
- protected com.twitter.distributedlog.DistributedLogManagerFactory factory = null;
+ protected DistributedLogNamespace namespace = null;
protected PerDLCommand(String name, String description) {
super(name, description);
@@ -187,8 +189,8 @@
return runCmd();
} finally {
synchronized (this) {
- if (null != factory) {
- factory.close();
+ if (null != namespace) {
+ namespace.close();
}
}
}
@@ -252,35 +254,33 @@
this.force = force;
}
- protected synchronized com.twitter.distributedlog.DistributedLogManagerFactory getFactory() throws IOException {
- if (null == this.factory) {
- this.factory = new com.twitter.distributedlog.DistributedLogManagerFactory(getConf(), getUri());
- logger.info("Construct DLM : uri = {}", getUri());
- }
- return this.factory;
- }
-
protected DistributedLogNamespace getNamespace() throws IOException {
- return getFactory().getNamespace();
+ if (null == this.namespace) {
+ this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+ .uri(getUri())
+ .conf(getConf())
+ .build();
+ }
+ return this.namespace;
}
protected LogSegmentMetadataStore getLogSegmentMetadataStore() throws IOException {
- DistributedLogNamespace namespace = getFactory().getNamespace();
- assert(namespace instanceof BKDistributedLogNamespace);
- return ((BKDistributedLogNamespace) namespace).getWriterStreamMetadataStore()
+ return getNamespace()
+ .getNamespaceDriver()
+ .getLogStreamMetadataStore(NamespaceDriver.Role.READER)
.getLogSegmentMetadataStore();
}
protected ZooKeeperClient getZooKeeperClient() throws IOException {
- DistributedLogNamespace namespace = getFactory().getNamespace();
- assert(namespace instanceof BKDistributedLogNamespace);
- return ((BKDistributedLogNamespace) namespace).getSharedWriterZKCForDL();
+ NamespaceDriver driver = getNamespace().getNamespaceDriver();
+ assert(driver instanceof BKNamespaceDriver);
+ return ((BKNamespaceDriver) driver).getWriterZKC();
}
protected BookKeeperClient getBookKeeperClient() throws IOException {
- DistributedLogNamespace namespace = getFactory().getNamespace();
- assert(namespace instanceof BKDistributedLogNamespace);
- return ((BKDistributedLogNamespace) namespace).getReaderBKC();
+ NamespaceDriver driver = getNamespace().getNamespaceDriver();
+ assert(driver instanceof BKNamespaceDriver);
+ return ((BKNamespaceDriver) driver).getReaderBKC();
}
}
@@ -347,6 +347,10 @@
}
}
+ /**
+ * NOTE: we might consider adding a command to 'delete' namespace. The implementation of the namespace
+ * driver should implement the 'delete' operation.
+ */
protected static class DeleteAllocatorPoolCommand extends PerDLCommand {
int concurrency = 1;
@@ -380,8 +384,12 @@
String rootPath = getUri().getPath() + "/" + allocationPoolPath;
final ScheduledExecutorService allocationExecutor = Executors.newSingleThreadScheduledExecutor();
ExecutorService executorService = Executors.newFixedThreadPool(concurrency);
+ Preconditions.checkArgument(getNamespace() instanceof BKDistributedLogNamespace);
+ BKDistributedLogNamespace bkns = (BKDistributedLogNamespace) getNamespace();
+ final ZooKeeperClient zkc = ((BKNamespaceDriver) bkns.getNamespaceDriver()).getWriterZKC();
+ final BookKeeperClient bkc = ((BKNamespaceDriver) bkns.getNamespaceDriver()).getReaderBKC();
try {
- List<String> pools = getZooKeeperClient().get().getChildren(rootPath, false);
+ List<String> pools = zkc.get().getChildren(rootPath, false);
final LinkedBlockingQueue<String> poolsToDelete = new LinkedBlockingQueue<String>();
if (getForce() || IOUtils.confirmPrompt("Are you sure you want to delete allocator pools : " + pools)) {
for (String pool : pools) {
@@ -401,7 +409,7 @@
try {
LedgerAllocator allocator =
LedgerAllocatorUtils.createLedgerAllocatorPool(poolPath, 0, getConf(),
- getZooKeeperClient(), getBookKeeperClient(),
+ zkc, bkc,
allocationExecutor);
allocator.delete();
System.out.println("Deleted allocator pool : " + poolPath + " .");
@@ -454,43 +462,35 @@
@Override
protected int runCmd() throws Exception {
- if (printMetadata) {
- printStreamsWithMetadata(getFactory());
- } else {
- printStreams(getFactory());
- }
+ printStreams(getNamespace());
return 0;
}
- protected void printStreamsWithMetadata(com.twitter.distributedlog.DistributedLogManagerFactory factory)
- throws Exception {
- Map<String, byte[]> streams = factory.enumerateLogsWithMetadataInNamespace();
+ protected void printStreams(DistributedLogNamespace namespace) throws Exception {
+ Iterator<String> streams = namespace.getLogs();
System.out.println("Streams under " + getUri() + " : ");
System.out.println("--------------------------------");
- for (Map.Entry<String, byte[]> entry : streams.entrySet()) {
- println(entry.getKey());
- if (null == entry.getValue() || entry.getValue().length == 0) {
+ while (streams.hasNext()) {
+ String streamName = streams.next();
+ System.out.println(streamName);
+ if (!printMetadata) {
+ continue;
+ }
+ MetadataAccessor accessor =
+ namespace.getNamespaceDriver().getMetadataAccessor(streamName);
+ byte[] metadata = accessor.getMetadata();
+ if (null == metadata || metadata.length == 0) {
continue;
}
if (printHex) {
- System.out.println(Hex.encodeHexString(entry.getValue()));
+ System.out.println(Hex.encodeHexString(metadata));
} else {
- System.out.println(new String(entry.getValue(), UTF_8));
+ System.out.println(new String(metadata, UTF_8));
}
System.out.println("");
}
System.out.println("--------------------------------");
}
-
- protected void printStreams(com.twitter.distributedlog.DistributedLogManagerFactory factory) throws Exception {
- Collection<String> streams = factory.enumerateAllLogsInNamespace();
- System.out.println("Streams under " + getUri() + " : ");
- System.out.println("--------------------------------");
- for (String stream : streams) {
- System.out.println(stream);
- }
- System.out.println("--------------------------------");
- }
}
public static class WatchNamespaceCommand extends PerDLCommand implements NamespaceListener {
@@ -609,16 +609,17 @@
private void inspectStreams(final SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates)
throws Exception {
- Collection<String> streamCollection = getFactory().enumerateAllLogsInNamespace();
+ Iterator<String> streamCollection = getNamespace().getLogs();
final List<String> streams = new ArrayList<String>();
- if (null != streamPrefix) {
- for (String s : streamCollection) {
+ while (streamCollection.hasNext()) {
+ String s = streamCollection.next();
+ if (null != streamPrefix) {
if (s.startsWith(streamPrefix)) {
streams.add(s);
}
+ } else {
+ streams.add(s);
}
- } else {
- streams.addAll(streamCollection);
}
if (0 == streams.size()) {
return;
@@ -660,8 +661,7 @@
for (int i = startIdx; i < endIdx; i++) {
String s = streams.get(i);
BookKeeperClient bkc = getBookKeeperClient();
- DistributedLogManager dlm =
- getFactory().createDistributedLogManagerWithSharedClients(s);
+ DistributedLogManager dlm = getNamespace().openLog(s);
try {
List<LogSegmentMetadata> segments = dlm.getLogSegments();
if (segments.size() <= 1) {
@@ -782,20 +782,21 @@
@Override
protected int runCmd() throws Exception {
getConf().setZkAclId(getZkAclId());
- return truncateStreams(getFactory());
+ return truncateStreams(getNamespace());
}
- private int truncateStreams(final com.twitter.distributedlog.DistributedLogManagerFactory factory) throws Exception {
- Collection<String> streamCollection = factory.enumerateAllLogsInNamespace();
+ private int truncateStreams(final DistributedLogNamespace namespace) throws Exception {
+ Iterator<String> streamCollection = namespace.getLogs();
final List<String> streams = new ArrayList<String>();
- if (null != streamPrefix) {
- for (String s : streamCollection) {
+ while (streamCollection.hasNext()) {
+ String s = streamCollection.next();
+ if (null != streamPrefix) {
if (s.startsWith(streamPrefix)) {
streams.add(s);
}
+ } else {
+ streams.add(s);
}
- } else {
- streams.addAll(streamCollection);
}
if (0 == streams.size()) {
return 0;
@@ -813,7 +814,7 @@
@Override
public void run() {
try {
- truncateStreams(factory, streams, tid, numStreamsPerThreads);
+ truncateStreams(namespace, streams, tid, numStreamsPerThreads);
System.out.println("Thread " + tid + " finished.");
} catch (IOException e) {
System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
@@ -828,14 +829,13 @@
return 0;
}
- private void truncateStreams(com.twitter.distributedlog.DistributedLogManagerFactory factory, List<String> streams,
+ private void truncateStreams(DistributedLogNamespace namespace, List<String> streams,
int tid, int numStreamsPerThreads) throws IOException {
int startIdx = tid * numStreamsPerThreads;
int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
for (int i = startIdx; i < endIdx; i++) {
String s = streams.get(i);
- DistributedLogManager dlm =
- factory.createDistributedLogManagerWithSharedClients(s);
+ DistributedLogManager dlm = namespace.openLog(s);
try {
if (deleteStream) {
dlm.delete();
@@ -930,7 +930,7 @@
@Override
protected int runCmd() throws Exception {
- DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+ DistributedLogManager dlm = getNamespace().openLog(getStreamName());
try {
if (listEppStats) {
bkc = new SimpleBookKeeperClient(getConf(), getUri());
@@ -1078,7 +1078,7 @@
@Override
protected int runCmd() throws Exception {
- DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+ DistributedLogManager dlm = getNamespace().openLog(getStreamName());
try {
long count = 0;
if (null == endDLSN) {
@@ -1141,7 +1141,7 @@
@Override
protected int runCmd() throws Exception {
getConf().setZkAclId(getZkAclId());
- DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+ DistributedLogManager dlm = getNamespace().openLog(getStreamName());
try {
dlm.delete();
} finally {
@@ -1347,7 +1347,7 @@
}
getConf().setZkAclId(getZkAclId());
for (String stream : streams) {
- getFactory().getNamespace().createLog(stream);
+ getNamespace().createLog(stream);
}
return 0;
}
@@ -1435,7 +1435,7 @@
@Override
protected int runCmd() throws Exception {
- DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+ DistributedLogManager dlm = getNamespace().openLog(getStreamName());
long totalCount = dlm.getLogRecordCount();
try {
AsyncLogReader reader;
@@ -1536,7 +1536,7 @@
@Override
protected int runCmd() throws Exception {
- DistributedLogManager dlm = getFactory().createDistributedLogManagerWithSharedClients(getStreamName());
+ DistributedLogManager dlm = getNamespace().openLog(getStreamName());
try {
return inspectAndRepair(dlm.getLogSegments());
} finally {
@@ -2640,11 +2640,11 @@
@Override
protected int runCmd() throws Exception {
getConf().setZkAclId(getZkAclId());
- return truncateStream(getFactory(), getStreamName(), dlsn);
+ return truncateStream(getNamespace(), getStreamName(), dlsn);
}
- private int truncateStream(final com.twitter.distributedlog.DistributedLogManagerFactory factory, String streamName, DLSN dlsn) throws Exception {
- DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+ private int truncateStream(final DistributedLogNamespace namespace, String streamName, DLSN dlsn) throws Exception {
+ DistributedLogManager dlm = namespace.openLog(streamName);
try {
long totalRecords = dlm.getLogRecordCount();
long recordsAfterTruncate = Await.result(dlm.getLogRecordCountAsync(dlsn));
@@ -2731,7 +2731,6 @@
int numThreads = 1;
String streamPrefix = null;
String subscriberId = null;
- AtomicInteger streamIndex = new AtomicInteger();
DeleteSubscriberCommand() {
super("delete_subscriber", "Delete the subscriber in subscription store. ");
@@ -2764,20 +2763,21 @@
@Override
protected int runCmd() throws Exception {
getConf().setZkAclId(getZkAclId());
- return deleteSubscriber(getFactory());
+ return deleteSubscriber(getNamespace());
}
- private int deleteSubscriber(final com.twitter.distributedlog.DistributedLogManagerFactory factory) throws Exception {
- Collection<String> streamCollection = factory.enumerateAllLogsInNamespace();
+ private int deleteSubscriber(final DistributedLogNamespace namespace) throws Exception {
+ Iterator<String> streamCollection = namespace.getLogs();
final List<String> streams = new ArrayList<String>();
- if (null != streamPrefix) {
- for (String s : streamCollection) {
+ while (streamCollection.hasNext()) {
+ String s = streamCollection.next();
+ if (null != streamPrefix) {
if (s.startsWith(streamPrefix)) {
streams.add(s);
}
+ } else {
+ streams.add(s);
}
- } else {
- streams.addAll(streamCollection);
}
if (0 == streams.size()) {
return 0;
@@ -2796,7 +2796,7 @@
@Override
public void run() {
try {
- deleteSubscriber(factory, streams, tid, numStreamsPerThreads);
+ deleteSubscriber(namespace, streams, tid, numStreamsPerThreads);
System.out.println("Thread " + tid + " finished.");
} catch (Exception e) {
System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
@@ -2811,14 +2811,13 @@
return 0;
}
- private void deleteSubscriber(com.twitter.distributedlog.DistributedLogManagerFactory factory, List<String> streams,
+ private void deleteSubscriber(DistributedLogNamespace namespace, List<String> streams,
int tid, int numStreamsPerThreads) throws Exception {
int startIdx = tid * numStreamsPerThreads;
int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
for (int i = startIdx; i < endIdx; i++) {
final String s = streams.get(i);
- DistributedLogManager dlm =
- factory.createDistributedLogManagerWithSharedClients(s);
+ DistributedLogManager dlm = namespace.openLog(s);
final CountDownLatch countDownLatch = new CountDownLatch(1);
dlm.getSubscriptionsStore().deleteSubscriber(subscriberId)
.addEventListener(new FutureEventListener<Boolean>() {
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
index 63db1fe..2f9e091 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
@@ -17,34 +17,27 @@
*/
package com.twitter.distributedlog.util;
+import com.google.common.base.Objects;
+import com.twitter.distributedlog.DistributedLogConstants;
import com.twitter.distributedlog.LogSegmentMetadata;
+import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
import com.twitter.distributedlog.exceptions.UnexpectedException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.commons.lang.StringUtils;
+import java.net.InetAddress;
import java.net.URI;
+import java.net.URISyntaxException;
import java.util.List;
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.
*/
public class DLUtils {
- static final Logger logger = LoggerFactory.getLogger(DLUtils.class);
-
- /**
- * Extract zk servers fro dl <i>uri</i>.
- *
- * @param uri
- * dl uri
- * @return zk servers
- */
- public static String getZKServersFromDLUri(URI uri) {
- return uri.getAuthority().replace(";", ",");
- }
-
/**
* Find the log segment whose transaction ids are not less than provided <code>transactionId</code>.
*
@@ -224,4 +217,105 @@
public static long bytes2LogSegmentId(byte[] data) {
return Long.parseLong(new String(data, UTF_8));
}
+
+ /**
+ * Normalize the uri.
+ *
+ * @param uri the distributedlog uri.
+ * @return the normalized uri
+ */
+ public static URI normalizeURI(URI uri) {
+ checkNotNull(uri, "DistributedLog uri is null");
+ String scheme = uri.getScheme();
+ checkNotNull(scheme, "Invalid distributedlog uri : " + uri);
+ scheme = scheme.toLowerCase();
+ String[] schemeParts = StringUtils.split(scheme, '-');
+ checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
+ "Unknown distributedlog scheme found : " + uri);
+ URI normalizedUri;
+ try {
+ normalizedUri = new URI(
+ schemeParts[0], // remove backend info
+ uri.getAuthority(),
+ uri.getPath(),
+ uri.getQuery(),
+ uri.getFragment());
+ } catch (URISyntaxException e) {
+ throw new IllegalArgumentException("Invalid distributedlog uri found : " + uri, e);
+ }
+ return normalizedUri;
+ }
+
+ private static String getHostIpLockClientId() {
+ try {
+ return InetAddress.getLocalHost().toString();
+ } catch(Exception ex) {
+ return DistributedLogConstants.UNKNOWN_CLIENT_ID;
+ }
+ }
+
+ /**
+ * Normalize the client id.
+ *
+ * @return the normalized client id.
+ */
+ public static String normalizeClientId(String clientId) {
+ String normalizedClientId;
+ if (clientId.equals(DistributedLogConstants.UNKNOWN_CLIENT_ID)) {
+ normalizedClientId = getHostIpLockClientId();
+ } else {
+ normalizedClientId = clientId;
+ }
+ return normalizedClientId;
+ }
+
+ /**
+ * Is it a reserved stream name in bkdl namespace?
+ *
+ * @param name
+ * stream name
+ * @return true if it is reserved name, otherwise false.
+ */
+ public static boolean isReservedStreamName(String name) {
+ return name.startsWith(".");
+ }
+
+ /**
+ * Validate the stream name.
+ *
+ * @param nameOfStream
+ * name of stream
+ * @throws InvalidStreamNameException
+ */
+ public static void validateName(String nameOfStream)
+ throws InvalidStreamNameException {
+ String reason = null;
+ char chars[] = nameOfStream.toCharArray();
+ char c;
+ // validate the stream to see if meet zookeeper path's requirement
+ for (int i = 0; i < chars.length; i++) {
+ c = chars[i];
+
+ if (c == 0) {
+ reason = "null character not allowed @" + i;
+ break;
+ } else if (c == '/') {
+ reason = "'/' not allowed @" + i;
+ break;
+ } else if (c > '\u0000' && c < '\u001f'
+ || c > '\u007f' && c < '\u009F'
+ || c > '\ud800' && c < '\uf8ff'
+ || c > '\ufff0' && c < '\uffff') {
+ reason = "invalid charater @" + i;
+ break;
+ }
+ }
+ if (null != reason) {
+ throw new InvalidStreamNameException(nameOfStream, reason);
+ }
+ if (isReservedStreamName(nameOfStream)) {
+ throw new InvalidStreamNameException(nameOfStream,
+ "Stream Name is reserved");
+ }
+ }
}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
index 266409e..f206a25 100644
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
+++ b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
@@ -271,7 +271,7 @@
*
* @param throwable the cause of the exception
* @return the bk exception return code. if the exception isn't bk exceptions,
- * it would return bk exception code.
+ * it would return {@link BKException.Code#UnexpectedConditionException}.
*/
public static int bkResultCode(Throwable throwable) {
if (throwable instanceof BKException) {
@@ -455,13 +455,13 @@
* @param key submit key of the ordered scheduler
*/
public static <T> void setException(final Promise<T> promise,
- final Throwable throwable,
+ final Throwable cause,
OrderedScheduler scheduler,
Object key) {
scheduler.submit(key, new Runnable() {
@Override
public void run() {
- setException(promise, throwable);
+ setException(promise, cause);
}
});
}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java
index b0a38cf..c403e26 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java
@@ -17,12 +17,12 @@
*/
package com.twitter.distributedlog;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.DLMetadata;
+import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
import com.twitter.distributedlog.util.ConfUtils;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.PermitLimiter;
@@ -35,11 +35,7 @@
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.versioning.Version;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -89,11 +85,6 @@
return segments;
}
- static void updateBKDLConfig(URI uri, String zkServers, String ledgersPath, boolean sanityCheckTxnID) throws Exception {
- BKDLConfig bkdlConfig = new BKDLConfig(zkServers, ledgersPath).setSanityCheckTxnID(sanityCheckTxnID);
- DLMetadata.create(bkdlConfig).update(uri);
- }
-
public static URI createDLMURI(int port, String path) throws Exception {
return LocalDLMEmulator.createDLMURI("127.0.0.1:" + port, path);
}
@@ -111,93 +102,18 @@
URI uri) throws Exception {
// TODO: Metadata Accessor seems to be a legacy object which only used by kestrel
// (we might consider deprecating this)
- BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
.conf(conf).uri(uri).build();
- return namespace.createMetadataAccessor(name);
- }
-
- public static class BKLogPartitionWriteHandlerAndClients {
- private BKLogWriteHandler writeHandler;
- private ZooKeeperClient zooKeeperClient;
- private BookKeeperClient bookKeeperClient;
-
- public BKLogPartitionWriteHandlerAndClients(BKLogWriteHandler writeHandler, ZooKeeperClient zooKeeperClient, BookKeeperClient bookKeeperClient) {
- this.writeHandler = writeHandler;
- this.zooKeeperClient = zooKeeperClient;
- this.bookKeeperClient = bookKeeperClient;
- }
-
- public void close() {
- bookKeeperClient.close();
- zooKeeperClient.close();
- Utils.closeQuietly(writeHandler);
- }
-
- public BKLogWriteHandler getWriteHandler() {
- return writeHandler;
- }
- }
-
- static BKLogPartitionWriteHandlerAndClients createNewBKDLM(DistributedLogConfiguration conf,
- String logName,
- int zkPort) throws Exception {
- URI uri = createDLMURI(zkPort, "/" + logName);
-
- ZooKeeperClientBuilder zkcBuilder = TestZooKeeperClientBuilder.newBuilder(conf)
- .name(String.format("dlzk:%s:handler_dedicated", logName))
- .uri(uri);
-
- ZooKeeperClient zkClient = zkcBuilder.build();
-
- try {
- zkClient.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- } catch (KeeperException.NodeExistsException nee) {
- // ignore
- }
-
- // resolve uri
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkClient, uri);
- BKDLConfig.propagateConfiguration(bkdlConfig, conf);
- BookKeeperClientBuilder bkcBuilder = BookKeeperClientBuilder.newBuilder()
- .dlConfig(conf)
- .name(String.format("bk:%s:handler_dedicated", logName))
- .zkServers(bkdlConfig.getBkZkServersForWriter())
- .ledgersPath(bkdlConfig.getBkLedgersPath())
- .statsLogger(NullStatsLogger.INSTANCE);
-
- BKDistributedLogManager bkdlm = new BKDistributedLogManager(
- logName,
- conf,
- uri,
- zkcBuilder,
- zkcBuilder,
- zkClient,
- zkClient,
- bkcBuilder,
- bkcBuilder,
- new SettableFeatureProvider("", 0),
- PermitLimiter.NULL_PERMIT_LIMITER,
- NullStatsLogger.INSTANCE);
-
- BKLogWriteHandler writeHandler = bkdlm.createWriteHandler(true);
- return new BKLogPartitionWriteHandlerAndClients(writeHandler, zkClient, bkcBuilder.build());
+ return namespace.getNamespaceDriver().getMetadataAccessor(name);
}
public static void fenceStream(DistributedLogConfiguration conf, URI uri, String name) throws Exception {
- BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(name, conf, uri);
+ DistributedLogManager dlm = createNewDLM(name, conf, uri);
try {
- BKLogReadHandler readHandler = dlm.createReadHandler();
- List<LogSegmentMetadata> ledgerList = FutureUtils.result(
- readHandler.readLogSegmentsFromStore(
- LogSegmentMetadata.COMPARATOR,
- LogSegmentFilter.DEFAULT_FILTER,
- null)
- ).getValue();
- LogSegmentMetadata lastSegment = ledgerList.get(ledgerList.size() - 1);
- BookKeeperClient bkc = dlm.getWriterBKC();
- LedgerHandle lh = bkc.get().openLedger(lastSegment.getLogSegmentId(),
- BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
- lh.close();
+ List<LogSegmentMetadata> logSegmentList = dlm.getLogSegments();
+ LogSegmentMetadata lastSegment = logSegmentList.get(logSegmentList.size() - 1);
+ LogSegmentEntryStore entryStore = dlm.getNamespaceDriver().getLogSegmentEntryStore(NamespaceDriver.Role.READER);
+ Utils.close(FutureUtils.result(entryStore.openRandomAccessReader(lastSegment, true)));
} finally {
dlm.close();
}
@@ -409,6 +325,14 @@
return txid - startTxid;
}
+ public static ZooKeeperClient getZooKeeperClient(BKDistributedLogManager dlm) {
+ return ((BKNamespaceDriver) dlm.getNamespaceDriver()).getWriterZKC();
+ }
+
+ public static BookKeeperClient getBookKeeperClient(BKDistributedLogManager dlm) {
+ return ((BKNamespaceDriver) dlm.getNamespaceDriver()).getReaderBKC();
+ }
+
public static void injectLogSegmentWithGivenLogSegmentSeqNo(DistributedLogManager manager, DistributedLogConfiguration conf,
long logSegmentSeqNo, long startTxID, boolean writeEntries, long segmentSize,
boolean completeLogSegment)
@@ -417,7 +341,7 @@
BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
FutureUtils.result(writeHandler.lockHandler());
// Start a log segment with a given ledger seq number.
- BookKeeperClient bkc = dlm.getWriterBKC();
+ BookKeeperClient bkc = getBookKeeperClient(dlm);
LedgerHandle lh = bkc.get().createLedger(conf.getEnsembleSize(), conf.getWriteQuorumSize(),
conf.getAckQuorumSize(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes());
String inprogressZnodeName = writeHandler.inprogressZNodeName(lh.getId(), startTxID, logSegmentSeqNo);
@@ -429,7 +353,7 @@
.setLogSegmentSequenceNo(logSegmentSeqNo)
.setEnvelopeEntries(LogSegmentMetadata.supportsEnvelopedEntries(logSegmentMetadataVersion))
.build();
- l.write(dlm.writerZKC);
+ l.write(getZooKeeperClient(dlm));
writeHandler.maxTxId.update(Version.ANY, startTxID);
writeHandler.addLogSegmentToCache(inprogressZnodeName, l);
BKLogSegmentWriter writer = new BKLogSegmentWriter(
@@ -468,7 +392,7 @@
BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
FutureUtils.result(writeHandler.lockHandler());
// Start a log segment with a given ledger seq number.
- BookKeeperClient bkc = dlm.getReaderBKC();
+ BookKeeperClient bkc = getBookKeeperClient(dlm);
LedgerHandle lh = bkc.get().createLedger(conf.getEnsembleSize(), conf.getWriteQuorumSize(),
conf.getAckQuorumSize(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes());
String inprogressZnodeName = writeHandler.inprogressZNodeName(lh.getId(), startTxID, logSegmentSeqNo);
@@ -479,7 +403,7 @@
.setLogSegmentSequenceNo(logSegmentSeqNo)
.setInprogress(false)
.build();
- l.write(dlm.writerZKC);
+ l.write(getZooKeeperClient(dlm));
writeHandler.maxTxId.update(Version.ANY, startTxID);
writeHandler.addLogSegmentToCache(inprogressZnodeName, l);
BKLogSegmentWriter writer = new BKLogSegmentWriter(
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java
index a6cffbb..124ea77 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java
@@ -29,9 +29,11 @@
import com.twitter.distributedlog.exceptions.LockCancelledException;
import com.twitter.distributedlog.exceptions.LockingException;
import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.lock.LockClosedException;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
import com.twitter.distributedlog.subscription.SubscriptionsStore;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.Utils;
@@ -85,7 +87,8 @@
Utils.close(reader1);
// simulate a old stream created without readlock path
- writer.bkDistributedLogManager.getWriterZKC().get().delete(readLockPath, -1);
+ NamespaceDriver driver = dlm.getNamespaceDriver();
+ ((BKNamespaceDriver) driver).getWriterZKC().get().delete(readLockPath, -1);
Future<AsyncLogReader> futureReader2 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
AsyncLogReader reader2 = Await.result(futureReader2);
record = Await.result(reader2.readNext());
@@ -230,7 +233,7 @@
DistributedLogManager dlm1 = createNewDLM(conf, name);
Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
- Await.result(futureReader1);
+ AsyncLogReader reader1 = Await.result(futureReader1);
BKDistributedLogManager dlm2 = (BKDistributedLogManager) createNewDLM(conf, name);
Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
@@ -243,6 +246,7 @@
} catch (LockCancelledException ex) {
}
+ Utils.close(reader1);
dlm0.close();
dlm1.close();
}
@@ -250,16 +254,26 @@
@Test(timeout = 60000)
public void testReaderLockSessionExpires() throws Exception {
String name = runtime.getMethodName();
- DistributedLogManager dlm0 = createNewDLM(conf, name);
+ URI uri = createDLMURI("/" + name);
+ ensureURICreated(uri);
+ DistributedLogNamespace ns0 = DistributedLogNamespaceBuilder.newBuilder()
+ .conf(conf)
+ .uri(uri)
+ .build();
+ DistributedLogManager dlm0 = ns0.openLog(name);
BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
writer.write(DLMTestUtil.getLogRecordInstance(1L));
writer.write(DLMTestUtil.getLogRecordInstance(2L));
writer.closeAndComplete();
- DistributedLogManager dlm1 = createNewDLM(conf, name);
+ DistributedLogNamespace ns1 = DistributedLogNamespaceBuilder.newBuilder()
+ .conf(conf)
+ .uri(uri)
+ .build();
+ DistributedLogManager dlm1 = ns1.openLog(name);
Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
AsyncLogReader reader1 = Await.result(futureReader1);
- ZooKeeperClientUtils.expireSession(((BKDistributedLogManager)dlm1).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
@@ -276,7 +290,9 @@
Utils.close(reader1);
dlm0.close();
+ ns0.close();
dlm1.close();
+ ns1.close();
}
@Test(timeout = 60000)
@@ -511,8 +527,11 @@
Utils.close(Await.result(futureReader3));
dlm1.close();
+ namespace1.close();
dlm2.close();
+ namespace2.close();
dlm3.close();
+ namespace3.close();
executorService.shutdown();
}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java
index 41adbb9..46c8523 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java
@@ -36,6 +36,7 @@
import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
import com.twitter.distributedlog.exceptions.BKTransmitException;
import com.twitter.distributedlog.exceptions.LockingException;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.io.CompressionCodec;
import com.twitter.distributedlog.util.Utils;
import com.twitter.util.Promise;
@@ -1270,8 +1271,9 @@
BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
+ BKNamespaceDriver driver = (BKNamespaceDriver) dlm.getNamespaceDriver();
// fence the ledger
- dlm.getWriterBKC().get().openLedger(logWriter.getLogSegmentId(),
+ driver.getReaderBKC().get().openLedger(logWriter.getLogSegmentId(),
BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
try {
@@ -1313,8 +1315,9 @@
BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
+ BKNamespaceDriver driver = (BKNamespaceDriver) dlm.getNamespaceDriver();
// fence the ledger
- dlm.getWriterBKC().get().openLedger(logWriter.getLogSegmentId(),
+ driver.getReaderBKC().get().openLedger(logWriter.getLogSegmentId(),
BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
try {
@@ -1500,6 +1503,7 @@
confLocal.setImmediateFlushEnabled(true);
confLocal.setReadAheadBatchSize(1);
confLocal.setReadAheadMaxRecords(1);
+ confLocal.setReadLACLongPollTimeout(49);
confLocal.setReaderIdleWarnThresholdMillis(100);
confLocal.setReaderIdleErrorThresholdMillis(20000);
final DistributedLogManager dlm = createNewDLM(confLocal, name);
@@ -1976,7 +1980,7 @@
List<LogSegmentMetadata> segments = dlm.getLogSegments();
assertEquals(1, segments.size());
long ledgerId = segments.get(0).getLogSegmentId();
- LedgerHandle lh = ((BKDistributedLogNamespace) namespace).getReaderBKC()
+ 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());
@@ -1995,7 +1999,7 @@
segments = dlm.getLogSegments();
assertEquals(1, segments.size());
ledgerId = segments.get(0).getLogSegmentId();
- lh = ((BKDistributedLogNamespace) namespace).getReaderBKC()
+ 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());
@@ -2147,6 +2151,7 @@
confLocal.setImmediateFlushEnabled(false);
confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
confLocal.setPeriodicKeepAliveMilliSeconds(0);
+ confLocal.setReadLACLongPollTimeout(9);
confLocal.setReaderIdleWarnThresholdMillis(20);
confLocal.setReaderIdleErrorThresholdMillis(40);
@@ -2178,6 +2183,7 @@
confLocal.setImmediateFlushEnabled(false);
confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
confLocal.setPeriodicKeepAliveMilliSeconds(1000);
+ confLocal.setReadLACLongPollTimeout(999);
confLocal.setReaderIdleWarnThresholdMillis(2000);
confLocal.setReaderIdleErrorThresholdMillis(4000);
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java
index 96c33e2..f7d587d 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java
@@ -22,6 +22,7 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Random;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
@@ -32,6 +33,7 @@
import com.twitter.distributedlog.exceptions.LogNotFoundException;
import com.twitter.distributedlog.exceptions.LogReadException;
import com.twitter.distributedlog.impl.ZKLogSegmentMetadataStore;
+import com.twitter.distributedlog.io.Abortables;
import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.OrderedScheduler;
@@ -48,14 +50,12 @@
import com.twitter.distributedlog.exceptions.EndOfStreamException;
import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
import com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException;
import com.twitter.distributedlog.metadata.LogMetadata;
import com.twitter.distributedlog.metadata.MetadataUpdater;
import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.subscription.SubscriptionStateStore;
import com.twitter.distributedlog.subscription.SubscriptionsStore;
import com.twitter.util.Await;
import com.twitter.util.Duration;
@@ -67,6 +67,8 @@
public class TestBKDistributedLogManager extends TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestBKDistributedLogManager.class);
+ private static final Random RAND = new Random(System.currentTimeMillis());
+
@Rule
public TestName testNames = new TestName();
@@ -254,20 +256,6 @@
}
@Test(timeout = 60000)
- public void testTwoWriters() throws Exception {
- DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlm1 =
- createNewBKDLM(conf, "distrlog-dualWriter");
- try {
- createNewBKDLM(conf, "distrlog-dualWriter");
- fail("Shouldn't have been able to open the second writer");
- } catch (OwnershipAcquireFailedException ioe) {
- assertEquals(ioe.getCurrentOwner(), DistributedLogConstants.UNKNOWN_CLIENT_ID);
- }
-
- bkdlm1.close();
- }
-
- @Test(timeout = 60000)
public void testTwoWritersOnLockDisabled() throws Exception {
DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
confLocal.addConfiguration(conf);
@@ -468,11 +456,10 @@
writer.setReadyToFlush();
writer.flushAndSync();
writer.close();
- dlm.createOrUpdateMetadata(name.getBytes());
- assertEquals(name, new String(dlm.getMetadata()));
+ dlm.close();
URI uri = createDLMURI("/" + name);
- BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
.conf(conf).uri(uri).build();
assertTrue(namespace.logExists(name));
assertFalse(namespace.logExists("non-existent-log"));
@@ -490,9 +477,7 @@
}
assertEquals(1, logCount);
- for(Map.Entry<String, byte[]> logEntry: namespace.enumerateLogsWithMetadataInNamespace().entrySet()) {
- assertEquals(name, new String(logEntry.getValue()));
- }
+ namespace.close();
}
@Test(timeout = 60000)
@@ -507,28 +492,6 @@
}
@Test(timeout = 60000)
- @Deprecated
- public void testSubscriptionStateStore() throws Exception {
- String name = "distrlog-subscription-state";
- String subscriberId = "defaultSubscriber";
- DLSN commitPosition0 = new DLSN(4, 33, 5);
- DLSN commitPosition1 = new DLSN(4, 34, 5);
- DLSN commitPosition2 = new DLSN(5, 34, 5);
-
- DistributedLogManager dlm = createNewDLM(conf, name);
- SubscriptionStateStore store = dlm.getSubscriptionStateStore(subscriberId);
- assertEquals(Await.result(store.getLastCommitPosition()), DLSN.NonInclusiveLowerBound);
- Await.result(store.advanceCommitPosition(commitPosition1));
- assertEquals(Await.result(store.getLastCommitPosition()), commitPosition1);
- Await.result(store.advanceCommitPosition(commitPosition0));
- assertEquals(Await.result(store.getLastCommitPosition()), commitPosition1);
- Await.result(store.advanceCommitPosition(commitPosition2));
- assertEquals(Await.result(store.getLastCommitPosition()), commitPosition2);
- SubscriptionStateStore store1 = dlm.getSubscriptionStateStore(subscriberId);
- assertEquals(Await.result(store1.getLastCommitPosition()), commitPosition2);
- }
-
- @Test(timeout = 60000)
public void testSubscriptionsStore() throws Exception {
String name = "distrlog-subscriptions-store";
String subscriber0 = "subscriber-0";
@@ -732,24 +695,12 @@
reader.close();
}
- @Test(timeout = 60000)
+ @Test(timeout = 60000, expected = LogRecordTooLongException.class)
public void testMaxLogRecSize() throws Exception {
- DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients =
- createNewBKDLM(conf, "distrlog-maxlogRecSize");
- long txid = 1;
- BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1);
- boolean exceptionEncountered = false;
- try {
- LogRecord op = new LogRecord(txid, DLMTestUtil.repeatString(
- DLMTestUtil.repeatString("abcdefgh", 256), 512).getBytes());
- out.write(op);
- } catch (LogRecordTooLongException exc) {
- exceptionEncountered = true;
- } finally {
- FutureUtils.result(out.asyncClose());
- }
- bkdlmAndClients.close();
- assertTrue(exceptionEncountered);
+ DistributedLogManager dlm = createNewDLM(conf, "distrlog-maxlogRecSize");
+ AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+ FutureUtils.result(writer.write(new LogRecord(1L, DLMTestUtil.repeatString(
+ DLMTestUtil.repeatString("abcdefgh", 256), 512).getBytes())));
}
@Test(timeout = 60000)
@@ -757,25 +708,27 @@
DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
confLocal.loadConf(conf);
confLocal.setOutputBufferSize(1024 * 1024);
- DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients =
- createNewBKDLM(confLocal, "distrlog-transmissionSize");
- long txid = 1;
- BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1);
+ BKDistributedLogManager dlm =
+ createNewDLM(confLocal, "distrlog-transmissionSize");
+ AsyncLogWriter out = FutureUtils.result(dlm.openAsyncLogWriter());
boolean exceptionEncountered = false;
- byte[] largePayload = DLMTestUtil.repeatString(DLMTestUtil.repeatString("abcdefgh", 256), 256).getBytes();
+ byte[] largePayload = new byte[(LogRecord.MAX_LOGRECORDSET_SIZE / 2) + 2];
+ RAND.nextBytes(largePayload);
try {
- while (txid < 3) {
- LogRecord op = new LogRecord(txid, largePayload);
- out.write(op);
- txid++;
- }
+ LogRecord op = new LogRecord(1L, largePayload);
+ Future<DLSN> firstWriteFuture = out.write(op);
+ op = new LogRecord(2L, largePayload);
+ // the second write will flush the first one, since we reached the maximum transmission size.
+ out.write(op);
+ FutureUtils.result(firstWriteFuture);
} catch (LogRecordTooLongException exc) {
exceptionEncountered = true;
} finally {
FutureUtils.result(out.asyncClose());
}
- bkdlmAndClients.close();
- assertTrue(!exceptionEncountered);
+ assertFalse(exceptionEncountered);
+ Abortables.abortQuietly(out);
+ dlm.close();
}
@Test(timeout = 60000)
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java
index ecc20e0..a8a82fa 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java
@@ -34,9 +34,10 @@
import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
import com.twitter.distributedlog.exceptions.LockingException;
import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.impl.BKDLUtils;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.util.DLUtils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
@@ -112,63 +113,13 @@
}
@Test(timeout = 60000)
- @SuppressWarnings("deprecation")
- public void testClientSharingOptions() throws Exception {
- URI uri = createDLMURI("/clientSharingOptions");
- BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
- .conf(conf).uri(uri).build();
-
- {
- BKDistributedLogManager bkdlm1 = (BKDistributedLogManager)namespace.createDistributedLogManager("perstream1",
- DistributedLogManagerFactory.ClientSharingOption.PerStreamClients);
-
- BKDistributedLogManager bkdlm2 = (BKDistributedLogManager)namespace.createDistributedLogManager("perstream2",
- DistributedLogManagerFactory.ClientSharingOption.PerStreamClients);
-
- assertThat(bkdlm1.getReaderBKC(), not(bkdlm2.getReaderBKC()));
- assertThat(bkdlm1.getWriterBKC(), not(bkdlm2.getWriterBKC()));
- assertThat(bkdlm1.getReaderZKC(), not(bkdlm2.getReaderZKC()));
- assertThat(bkdlm1.getWriterZKC(), not(bkdlm2.getWriterZKC()));
-
- }
-
- {
- BKDistributedLogManager bkdlm1 = (BKDistributedLogManager)namespace.createDistributedLogManager("sharedZK1",
- DistributedLogManagerFactory.ClientSharingOption.SharedZKClientPerStreamBKClient);
-
- BKDistributedLogManager bkdlm2 = (BKDistributedLogManager)namespace.createDistributedLogManager("sharedZK2",
- DistributedLogManagerFactory.ClientSharingOption.SharedZKClientPerStreamBKClient);
-
- assertThat(bkdlm1.getReaderBKC(), not(bkdlm2.getReaderBKC()));
- assertThat(bkdlm1.getWriterBKC(), not(bkdlm2.getWriterBKC()));
- assertEquals(bkdlm1.getReaderZKC(), bkdlm2.getReaderZKC());
- assertEquals(bkdlm1.getWriterZKC(), bkdlm2.getWriterZKC());
- }
-
- {
- BKDistributedLogManager bkdlm1 = (BKDistributedLogManager)namespace.createDistributedLogManager("sharedBoth1",
- DistributedLogManagerFactory.ClientSharingOption.SharedClients);
-
- BKDistributedLogManager bkdlm2 = (BKDistributedLogManager)namespace.createDistributedLogManager("sharedBoth2",
- DistributedLogManagerFactory.ClientSharingOption.SharedClients);
-
- assertEquals(bkdlm1.getReaderBKC(), bkdlm2.getReaderBKC());
- assertEquals(bkdlm1.getWriterBKC(), bkdlm2.getWriterBKC());
- assertEquals(bkdlm1.getReaderZKC(), bkdlm2.getReaderZKC());
- assertEquals(bkdlm1.getWriterZKC(), bkdlm2.getWriterZKC());
- }
-
- }
-
-
- @Test(timeout = 60000)
public void testInvalidStreamName() throws Exception {
- assertFalse(BKDLUtils.isReservedStreamName("test"));
- assertTrue(BKDLUtils.isReservedStreamName(".test"));
+ assertFalse(DLUtils.isReservedStreamName("test"));
+ assertTrue(DLUtils.isReservedStreamName(".test"));
URI uri = createDLMURI("/" + runtime.getMethodName());
- BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
.conf(conf).uri(uri).build();
try {
@@ -238,11 +189,6 @@
assertTrue(streamSet.contains("test1"));
assertTrue(streamSet.contains("test_2-3"));
- Map<String, byte[]> streamMetadatas = namespace.enumerateLogsWithMetadataInNamespace();
- assertEquals(2, streamMetadatas.size());
- assertTrue(streamMetadatas.containsKey("test1"));
- assertTrue(streamMetadatas.containsKey("test_2-3"));
-
namespace.close();
}
@@ -385,7 +331,7 @@
static void validateBadAllocatorConfiguration(DistributedLogConfiguration conf, URI uri) throws Exception {
try {
- BKDistributedLogNamespace.validateAndGetFullLedgerAllocatorPoolPath(conf, uri);
+ BKNamespaceDriver.validateAndGetFullLedgerAllocatorPoolPath(conf, uri);
fail("Should throw exception when bad allocator configuration provided");
} catch (IOException ioe) {
// expected
@@ -425,7 +371,7 @@
@Test(timeout = 60000)
public void testUseNamespaceAfterCloseShouldFailFast() throws Exception {
URI uri = createDLMURI("/" + runtime.getMethodName());
- BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
.conf(conf)
.uri(uri)
.build();
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java
index 4b17500..854cb74 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java
@@ -50,27 +50,6 @@
@Rule
public TestName runtime = new TestName();
- private void prepareLogSegments(String name, int numSegments, int numEntriesPerSegment) throws Exception {
- DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, name);
- long txid = 1;
- for (int sid = 0; sid < numSegments; ++sid) {
- BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(txid);
- for (int eid = 0; eid < numEntriesPerSegment; ++eid) {
- LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
- out.write(record);
- ++txid;
- }
- FutureUtils.result(out.asyncClose());
- bkdlmAndClients.getWriteHandler().completeAndCloseLogSegment(
- out.getLogSegmentSequenceNumber(),
- out.getLogSegmentId(),
- 1 + sid * numEntriesPerSegment,
- (sid + 1) * numEntriesPerSegment,
- numEntriesPerSegment);
- }
- bkdlmAndClients.close();
- }
-
private void prepareLogSegmentsNonPartitioned(String name, int numSegments, int numEntriesPerSegment) throws Exception {
DistributedLogManager dlm = createNewDLM(conf, name);
long txid = 1;
@@ -134,8 +113,8 @@
@Test(timeout = 60000)
public void testGetFirstDLSNWithLogSegments() throws Exception {
String dlName = runtime.getMethodName();
- prepareLogSegments(dlName, 3, 3);
BKDistributedLogManager dlm = createNewDLM(conf, dlName);
+ DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 3);
BKLogReadHandler readHandler = dlm.createReadHandler();
Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
try {
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java
index b350255..8f86192 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java
@@ -21,14 +21,14 @@
import com.twitter.distributedlog.exceptions.EndOfStreamException;
import com.twitter.distributedlog.exceptions.WriteCancelledException;
import com.twitter.distributedlog.exceptions.WriteException;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
import com.twitter.distributedlog.io.Abortables;
import com.twitter.distributedlog.lock.SessionLockFactory;
import com.twitter.distributedlog.lock.ZKDistributedLock;
import com.twitter.distributedlog.lock.ZKSessionLockFactory;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.DLUtils;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.OrderedScheduler;
import com.twitter.distributedlog.util.PermitLimiter;
@@ -96,7 +96,7 @@
.dlConfig(conf)
.name("test-bkc")
.ledgersPath(bkdlConfig.getBkLedgersPath())
- .zkServers(DLUtils.getZKServersFromDLUri(uri))
+ .zkServers(BKNamespaceDriver.getZKServersFromDLUri(uri))
.build();
}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java
index 754f945..a0485bd 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java
@@ -19,6 +19,7 @@
import com.twitter.distributedlog.bk.LedgerAllocator;
import com.twitter.distributedlog.bk.LedgerAllocatorPool;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
import com.twitter.distributedlog.util.FailpointUtils;
import com.twitter.distributedlog.util.FutureUtils;
@@ -75,7 +76,8 @@
FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber);
}
- LedgerAllocator allocator = namespace.getLedgerAllocator();
+ LedgerAllocator allocator = ((BKNamespaceDriver) namespace.getNamespaceDriver())
+ .getLedgerAllocator();
assertTrue(allocator instanceof LedgerAllocatorPool);
LedgerAllocatorPool allocatorPool = (LedgerAllocatorPool) allocator;
assertEquals(0, allocatorPool.obtainMapSize());
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java
index a388b68..d850db4 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java
@@ -19,11 +19,24 @@
import static org.junit.Assert.assertTrue;
+import com.google.common.base.Optional;
+import com.google.common.base.Ticker;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
+import com.twitter.distributedlog.injector.AsyncFailureInjector;
+import com.twitter.distributedlog.injector.AsyncRandomFailureInjector;
+import com.twitter.distributedlog.io.AsyncCloseable;
import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
+import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import com.twitter.distributedlog.namespace.NamespaceDriver;
+import com.twitter.distributedlog.util.ConfUtils;
+import com.twitter.distributedlog.util.OrderedScheduler;
import com.twitter.distributedlog.util.PermitLimiter;
+import com.twitter.distributedlog.util.SchedulerUtils;
+import com.twitter.util.Future;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.feature.SettableFeatureProvider;
import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
@@ -43,9 +56,11 @@
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;
public class TestDistributedLogBase {
static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogBase.class);
@@ -87,6 +102,12 @@
.build();
bkutil.start();
zkServers = "127.0.0.1:" + zkPort;
+ Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+ @Override
+ public void uncaughtException(Thread t, Throwable e) {
+ LOG.warn("Uncaught exception at Thread {} : ", t.getName(), e);
+ }
+ });
}
@AfterClass
@@ -141,22 +162,7 @@
public BKDistributedLogManager createNewDLM(DistributedLogConfiguration conf,
String name) throws Exception {
- URI uri = createDLMURI("/" + name);
- ensureURICreated(uri);
- return new BKDistributedLogManager(
- name,
- conf,
- uri,
- null,
- null,
- null,
- null,
- null,
- null,
- new SettableFeatureProvider("", 0),
- PermitLimiter.NULL_PERMIT_LIMITER,
- NullStatsLogger.INSTANCE
- );
+ return createNewDLM(conf, name, PermitLimiter.NULL_PERMIT_LIMITER);
}
public BKDistributedLogManager createNewDLM(DistributedLogConfiguration conf,
@@ -165,48 +171,69 @@
throws Exception {
URI uri = createDLMURI("/" + name);
ensureURICreated(uri);
+ final DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+ .uri(uri)
+ .conf(conf)
+ .build();
+ final OrderedScheduler scheduler = OrderedScheduler.newBuilder()
+ .corePoolSize(1)
+ .name("test-scheduler")
+ .build();
+ AsyncCloseable resourcesCloseable = new AsyncCloseable() {
+ @Override
+ public Future<Void> asyncClose() {
+ LOG.info("Shutting down the scheduler");
+ SchedulerUtils.shutdownScheduler(scheduler, 1, TimeUnit.SECONDS);
+ LOG.info("Shut down the scheduler");
+ LOG.info("Closing the namespace");
+ namespace.close();
+ LOG.info("Closed the namespace");
+ return Future.Void();
+ }
+ };
+ AsyncFailureInjector failureInjector = AsyncRandomFailureInjector.newBuilder()
+ .injectDelays(conf.getEIInjectReadAheadDelay(),
+ conf.getEIInjectReadAheadDelayPercent(),
+ conf.getEIInjectMaxReadAheadDelayMs())
+ .injectErrors(false, 10)
+ .injectStops(conf.getEIInjectReadAheadStall(), 10)
+ .injectCorruption(conf.getEIInjectReadAheadBrokenEntries())
+ .build();
return new BKDistributedLogManager(
name,
conf,
+ ConfUtils.getConstDynConf(conf),
uri,
- null,
- null,
- null,
- null,
- null,
- null,
- new SettableFeatureProvider("", 0),
+ namespace.getNamespaceDriver(),
+ new LogSegmentMetadataCache(conf, Ticker.systemTicker()),
+ scheduler,
+ DistributedLogConstants.UNKNOWN_CLIENT_ID,
+ DistributedLogConstants.LOCAL_REGION_ID,
writeLimiter,
- NullStatsLogger.INSTANCE
- );
+ new SettableFeatureProvider("", 0),
+ failureInjector,
+ NullStatsLogger.INSTANCE,
+ NullStatsLogger.INSTANCE,
+ Optional.of(resourcesCloseable));
}
- public DLMTestUtil.BKLogPartitionWriteHandlerAndClients createNewBKDLM(
- DistributedLogConfiguration conf,
- String path) throws Exception {
- return DLMTestUtil.createNewBKDLM(conf, path, zkPort);
- }
-
- @SuppressWarnings("deprecation")
- protected LogSegmentMetadataStore getLogSegmentMetadataStore(DistributedLogManagerFactory factory) {
- DistributedLogNamespace namespace = factory.getNamespace();
- assertTrue(namespace instanceof BKDistributedLogNamespace);
- return ((BKDistributedLogNamespace) namespace).getWriterStreamMetadataStore()
+ protected LogSegmentMetadataStore getLogSegmentMetadataStore(DistributedLogNamespace namespace)
+ throws IOException {
+ return namespace.getNamespaceDriver().getLogStreamMetadataStore(NamespaceDriver.Role.READER)
.getLogSegmentMetadataStore();
}
- @SuppressWarnings("deprecation")
- protected ZooKeeperClient getZooKeeperClient(DistributedLogManagerFactory factory) throws Exception {
- DistributedLogNamespace namespace = factory.getNamespace();
- assertTrue(namespace instanceof BKDistributedLogNamespace);
- return ((BKDistributedLogNamespace) namespace).getSharedWriterZKCForDL();
+ protected ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) throws Exception {
+ NamespaceDriver driver = namespace.getNamespaceDriver();
+ assertTrue(driver instanceof BKNamespaceDriver);
+ return ((BKNamespaceDriver) driver).getWriterZKC();
}
@SuppressWarnings("deprecation")
- protected BookKeeperClient getBookKeeperClient(DistributedLogManagerFactory factory) throws Exception {
- DistributedLogNamespace namespace = factory.getNamespace();
- assertTrue(namespace instanceof BKDistributedLogNamespace);
- return ((BKDistributedLogNamespace) namespace).getReaderBKC();
+ protected BookKeeperClient getBookKeeperClient(DistributedLogNamespace namespace) throws Exception {
+ NamespaceDriver driver = namespace.getNamespaceDriver();
+ assertTrue(driver instanceof BKNamespaceDriver);
+ return ((BKNamespaceDriver) driver).getReaderBKC();
}
protected LedgerHandle getLedgerHandle(BKLogSegmentWriter segmentWriter) {
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestFailureAndRecovery.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestFailureAndRecovery.java
deleted file mode 100644
index e86e45a..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestFailureAndRecovery.java
+++ /dev/null
@@ -1,257 +0,0 @@
-/**
- * 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.
- */
-package com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.io.Abortables;
-import com.twitter.distributedlog.util.FutureUtils;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.proto.BookieServer;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class TestFailureAndRecovery extends TestDistributedLogBase {
- static final Log LOG = LogFactory.getLog(TestFailureAndRecovery.class);
-
- @Test(timeout = 60000)
- public void testSimpleRecovery() throws Exception {
- DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, "distrlog-simplerecovery");
- BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1);
- long txid = 1;
- for (long i = 1; i <= 100; i++) {
- LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
- out.write(op);
- if ((i % 10) == 0) {
- FutureUtils.result(out.flushAndCommit());
- }
-
- }
- FutureUtils.result(out.flushAndCommit());
-
- Abortables.abort(out, false);
- FutureUtils.result(out.asyncClose());
-
- assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(1, 100, out.getLogSegmentSequenceNumber()), false));
- assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(out.getLogSegmentId(), 1, out.getLogSegmentSequenceNumber()), false));
-
- FutureUtils.result(bkdlmAndClients.getWriteHandler().recoverIncompleteLogSegments());
-
- assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(1, 100, out.getLogSegmentSequenceNumber()), false));
- assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(out.getLogSegmentId(), 1, out.getLogSegmentSequenceNumber()), false));
- }
-
- /**
- * Test that if enough bookies fail to prevent an ensemble,
- * writes the bookkeeper will fail. Test that when once again
- * an ensemble is available, it can continue to write.
- */
- @Test(timeout = 60000)
- public void testAllBookieFailure() throws Exception {
- BookieServer bookieToFail = bkutil.newBookie();
- BookieServer replacementBookie = null;
-
- try {
- int ensembleSize = numBookies + 1;
- assertEquals("Begin: New bookie didn't start",
- ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
- // ensure that the journal manager has to use all bookies,
- // so that a failure will fail the journal manager
- DistributedLogConfiguration conf = new DistributedLogConfiguration();
- conf.setEnsembleSize(ensembleSize);
- conf.setWriteQuorumSize(ensembleSize);
- conf.setAckQuorumSize(ensembleSize);
- long txid = 1;
- DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, "distrlog-allbookiefailure");
- BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(txid);
-
- for (long i = 1; i <= 3; i++) {
- LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
- out.write(op);
- }
- FutureUtils.result(out.flushAndCommit());
- bookieToFail.shutdown();
- assertEquals("New bookie didn't die",
- numBookies, bkutil.checkBookiesUp(numBookies, 10));
-
- try {
- for (long i = 1; i <= 3; i++) {
- LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
- out.write(op);
- txid++;
- }
- FutureUtils.result(out.flushAndCommit());
- fail("should not get to this stage");
- } catch (BKTransmitException bkte) {
- LOG.debug("Error writing to bookkeeper", bkte);
- assertEquals("Invalid exception message",
- BKException.Code.NotEnoughBookiesException, bkte.getBKResultCode());
- }
- replacementBookie = bkutil.newBookie();
-
- assertEquals("Replacement: New bookie didn't start",
- numBookies + 1, bkutil.checkBookiesUp(numBookies + 1, 10));
- out = bkdlmAndClients.getWriteHandler().startLogSegment(txid);
- for (long i = 1; i <= 3; i++) {
- LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
- out.write(op);
- }
-
- FutureUtils.result(out.flushAndCommit());
- } catch (Exception e) {
- LOG.error("Exception in test", e);
- throw e;
- } finally {
- if (replacementBookie != null) {
- replacementBookie.shutdown();
- }
- bookieToFail.shutdown();
-
- if (bkutil.checkBookiesUp(numBookies, 30) != numBookies) {
- LOG.warn("Not all bookies from this test shut down, expect errors");
- }
- }
- }
-
- /**
- * Test that a BookKeeper JM can continue to work across the
- * failure of a bookie. This should be handled transparently
- * by bookkeeper.
- */
- @Test(timeout = 60000)
- public void testOneBookieFailure() throws Exception {
- BookieServer bookieToFail = bkutil.newBookie();
- BookieServer replacementBookie = null;
-
- try {
- int ensembleSize = numBookies + 1;
- assertEquals("New bookie didn't start",
- ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
- // ensure that the journal manager has to use all bookies,
- // so that a failure will fail the journal manager
- DistributedLogConfiguration conf = new DistributedLogConfiguration();
- conf.setEnsembleSize(ensembleSize);
- conf.setWriteQuorumSize(ensembleSize);
- conf.setAckQuorumSize(ensembleSize);
- long txid = 1;
- DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, "distrlog-onebookiefailure");
- BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(txid);
- for (long i = 1; i <= 3; i++) {
- LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
- out.write(op);
- }
- FutureUtils.result(out.flushAndCommit());
-
- replacementBookie = bkutil.newBookie();
- assertEquals("replacement bookie didn't start",
- ensembleSize + 1, bkutil.checkBookiesUp(ensembleSize + 1, 10));
- bookieToFail.shutdown();
- assertEquals("New bookie didn't die",
- ensembleSize, bkutil.checkBookiesUp(ensembleSize, 10));
-
- for (long i = 1; i <= 3; i++) {
- LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
- out.write(op);
- }
- FutureUtils.result(out.flushAndCommit());
- } catch (Exception e) {
- LOG.error("Exception in test", e);
- throw e;
- } finally {
- if (replacementBookie != null) {
- replacementBookie.shutdown();
- }
- bookieToFail.shutdown();
-
- if (bkutil.checkBookiesUp(numBookies, 30) != numBookies) {
- LOG.warn("Not all bookies from this test shut down, expect errors");
- }
- }
- }
-
- @Test(timeout = 60000)
- public void testRecoveryEmptyLedger() throws Exception {
- DLMTestUtil.BKLogPartitionWriteHandlerAndClients bkdlmAndClients = createNewBKDLM(conf, "distrlog-recovery-empty-ledger");
- BKLogSegmentWriter out = bkdlmAndClients.getWriteHandler().startLogSegment(1);
- long txid = 1;
- for (long i = 1; i <= 100; i++) {
- LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
- out.write(op);
- if ((i % 10) == 0) {
- FutureUtils.result(out.flushAndCommit());
- }
-
- }
- FutureUtils.result(out.flushAndCommit());
- FutureUtils.result(out.asyncClose());
- bkdlmAndClients.getWriteHandler().completeAndCloseLogSegment(out.getLogSegmentSequenceNumber(), out.getLogSegmentId(), 1, 100, 100);
- assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(1, 100, out.getLogSegmentSequenceNumber()), false));
- BKLogSegmentWriter outEmpty = bkdlmAndClients.getWriteHandler().startLogSegment(101);
- Abortables.abort(outEmpty, false);
-
- assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(101, 101, outEmpty.getLogSegmentSequenceNumber()), false));
- assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(outEmpty.getLogSegmentId(), 101, outEmpty.getLogSegmentSequenceNumber()), false));
-
- FutureUtils.result(bkdlmAndClients.getWriteHandler().recoverIncompleteLogSegments());
-
- assertNull(zkc.exists(bkdlmAndClients.getWriteHandler().inprogressZNode(outEmpty.getLogSegmentId(), outEmpty.getLogSegmentSequenceNumber(), 101), false));
- assertNotNull(zkc.exists(bkdlmAndClients.getWriteHandler().completedLedgerZNode(101, 101, outEmpty.getLogSegmentSequenceNumber()), false));
- }
-
- @Test(timeout = 60000)
- public void testRecoveryAPI() throws Exception {
- DistributedLogManager dlm = createNewDLM(conf, "distrlog-recovery-api");
- BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
- long txid = 1;
- for (long i = 1; i <= 100; i++) {
- LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
- out.write(op);
- if ((i % 10) == 0) {
- out.setReadyToFlush();
- out.flushAndSync();
- }
-
- }
- BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
- out.setReadyToFlush();
- out.flushAndSync();
-
- out.abort();
-
- BKLogWriteHandler blplm1 = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
-
- assertNull(zkc.exists(blplm1.completedLedgerZNode(1, 100,
- perStreamLogWriter.getLogSegmentSequenceNumber()), false));
- assertNotNull(zkc.exists(blplm1.inprogressZNode(perStreamLogWriter.getLogSegmentId(), 1,
- perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-
- dlm.recover();
-
- assertNotNull(zkc.exists(blplm1.completedLedgerZNode(1, 100,
- perStreamLogWriter.getLogSegmentSequenceNumber()), false));
- assertNull(zkc.exists(blplm1.inprogressZNode(perStreamLogWriter.getLogSegmentId(), 1,
- perStreamLogWriter.getLogSegmentSequenceNumber()), false));
- FutureUtils.result(blplm1.asyncClose());
- assertEquals(100, dlm.getLogRecordCount());
- dlm.close();
- }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java
index 3cdd676..830e059 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java
@@ -22,8 +22,6 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.net.URI;
-
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
@@ -337,72 +335,4 @@
dlmreader1.close();
}
- @Test(timeout = 60000)
- public void testFactorySharedClients() throws Exception {
- String name = "distrlog-factorysharedclients";
- testFactory(name, true);
- }
-
- @Test(timeout = 60000)
- public void testFactorySharedZK() throws Exception {
- String name = "distrlog-factorysharedZK";
- testFactory(name, false);
- }
-
- @SuppressWarnings("deprecation")
- private void testFactory(String name, boolean shareBK) throws Exception {
- int count = 3;
- URI uri = createDLMURI("/" + name);
- ensureURICreated(uri);
- BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder()
- .conf(conf).uri(uri).build();
- DistributedLogManager[] dlms = new DistributedLogManager[count];
- for (int s = 0; s < count; s++) {
- if (shareBK) {
- dlms[s] = namespace.createDistributedLogManager(name + String.format("%d", s),
- DistributedLogManagerFactory.ClientSharingOption.SharedClients);
- } else {
- dlms[s] = namespace.createDistributedLogManager(name + String.format("%d", s),
- DistributedLogManagerFactory.ClientSharingOption.SharedZKClientPerStreamBKClient);
- }
- }
-
- int txid = 1;
- for (long i = 0; i < 3; i++) {
- BKSyncLogWriter[] writers = new BKSyncLogWriter[count];
- for (int s = 0; s < count; s++) {
- writers[s] = (BKSyncLogWriter)(dlms[s].startLogSegmentNonPartitioned());
- }
-
- for (long j = 0; j < 1; j++) {
- final LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
- for (int s = 0; s < count; s++) {
- writers[s].write(record);
- }
- }
- for (int s = 0; s < count; s++) {
- writers[s].closeAndComplete();
- }
-
- if (i < 2) {
- // Restart the zeroth stream and make sure that the other streams can
- // continue without restart
- dlms[0].close();
- if (shareBK) {
- dlms[0] = namespace.createDistributedLogManager(name + String.format("%d", 0),
- DistributedLogManagerFactory.ClientSharingOption.SharedClients);
- } else {
- dlms[0] = namespace.createDistributedLogManager(name + String.format("%d", 0),
- DistributedLogManagerFactory.ClientSharingOption.SharedZKClientPerStreamBKClient);
- }
- }
-
- }
-
- for (int s = 0; s < count; s++) {
- dlms[s].close();
- }
-
- namespace.close();
- }
}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java
index bb67214..06c7bba 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java
@@ -80,10 +80,10 @@
.setImmediateFlushEnabled(true)
.setEnableLedgerAllocatorPool(true)
.setLedgerAllocatorPoolName("test");
- BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder().conf(conf).uri(uri).build();
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
namespace.createLog(streamName);
- MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+ MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
DistributedLogManager dlm = namespace.openLog(streamName);
final int numSegments = 3;
@@ -92,7 +92,7 @@
out.write(DLMTestUtil.getLogRecordInstance(i));
out.closeAndComplete();
}
- MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+ MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
assertEquals(3, max2.getSequenceNumber());
dlm.close();
namespace.close();
@@ -111,10 +111,10 @@
.setImmediateFlushEnabled(true)
.setEnableLedgerAllocatorPool(true)
.setLedgerAllocatorPoolName("test");
- BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder().conf(conf).uri(uri).build();
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
namespace.createLog(streamName);
- MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+ MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
DistributedLogManager dlm = namespace.openLog(streamName);
final int numSegments = 3;
@@ -123,11 +123,11 @@
out.write(DLMTestUtil.getLogRecordInstance(i));
out.closeAndComplete();
}
- MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+ MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
assertEquals(3, max2.getSequenceNumber());
// nuke the max ledger sequence number
- updateMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf, new byte[0]);
+ updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf, new byte[0]);
DistributedLogManager dlm1 = namespace.openLog(streamName);
try {
dlm1.startLogSegmentNonPartitioned();
@@ -139,7 +139,7 @@
}
// invalid max ledger sequence number
- updateMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), 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();
@@ -167,10 +167,10 @@
.setImmediateFlushEnabled(true)
.setEnableLedgerAllocatorPool(true)
.setLedgerAllocatorPoolName("test");
- BKDistributedLogNamespace namespace = BKDistributedLogNamespace.newBuilder().conf(conf).uri(uri).build();
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
namespace.createLog(streamName);
- MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+ MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
DistributedLogManager dlm = namespace.openLog(streamName);
final int numSegments = 3;
@@ -179,11 +179,11 @@
out.write(DLMTestUtil.getLogRecordInstance(i));
out.closeAndComplete();
}
- MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf);
+ MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
assertEquals(3, max2.getSequenceNumber());
// update the max ledger sequence number
- updateMaxLogSegmentSequenceNo(namespace.getSharedWriterZKCForDL(), uri, streamName, conf,
+ updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf,
DLUtils.serializeLogSegmentSequenceNumber(99));
DistributedLogManager dlm1 = namespace.openLog(streamName);
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java
index e322234..9553637 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java
@@ -54,6 +54,7 @@
confLocal.setReadAheadBatchSize(1);
confLocal.setReadAheadMaxRecords(1);
confLocal.setReaderIdleWarnThresholdMillis(100);
+ confLocal.setReadLACLongPollTimeout(49);
final DistributedLogManager dlm = createNewDLM(confLocal, name);
ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
ScheduledFuture writerClosedFuture = null;
@@ -129,6 +130,7 @@
confLocal.loadConf(conf);
confLocal.setReadAheadBatchSize(1);
confLocal.setReadAheadMaxRecords(1);
+ confLocal.setReadLACLongPollTimeout(24);
confLocal.setReaderIdleWarnThresholdMillis(50);
confLocal.setReaderIdleErrorThresholdMillis(100);
final DistributedLogManager dlm = createNewDLM(confLocal, name);
@@ -174,6 +176,7 @@
confLocal.loadConf(conf);
confLocal.setReadAheadBatchSize(1);
confLocal.setReadAheadMaxRecords(3);
+ confLocal.setReadLACLongPollTimeout(249);
confLocal.setReaderIdleWarnThresholdMillis(500);
confLocal.setReaderIdleErrorThresholdMillis(30000);
final DistributedLogManager dlm = createNewDLM(confLocal, name);
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java
index 74a5231..cf4fc4f 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java
@@ -25,6 +25,7 @@
import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
import com.twitter.distributedlog.injector.AsyncFailureInjector;
import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.util.ConfUtils;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.OrderedScheduler;
import com.twitter.distributedlog.util.Utils;
@@ -38,6 +39,7 @@
import org.junit.rules.TestName;
import java.util.List;
+import java.util.concurrent.TimeUnit;
import static org.junit.Assert.*;
@@ -54,6 +56,7 @@
private DistributedLogConfiguration baseConf;
private OrderedScheduler scheduler;
private BookKeeperClient bkc;
+ private ZooKeeperClient zkc;
@Before
public void setup() throws Exception {
@@ -66,6 +69,12 @@
baseConf.setReadAheadMaxRecords(MAX_CACHED_ENTRIES);
baseConf.setNumPrefetchEntriesPerLogSegment(NUM_PREFETCH_ENTRIES);
baseConf.setMaxPrefetchEntriesPerLogSegment(NUM_PREFETCH_ENTRIES);
+ zkc = ZooKeeperClientBuilder.newBuilder()
+ .name("test-zk")
+ .zkServers(bkutil.getZkServers())
+ .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+ .zkAclId(conf.getZkAclId())
+ .build();
bkc = BookKeeperClientBuilder.newBuilder()
.name("test-bk")
.dlConfig(conf)
@@ -86,6 +95,9 @@
if (null != scheduler) {
scheduler.shutdown();
}
+ if (null != zkc) {
+ zkc.close();
+ }
super.teardown();
}
@@ -99,8 +111,11 @@
true);
LogSegmentEntryStore entryStore = new BKLogSegmentEntryStore(
conf,
+ ConfUtils.getConstDynConf(conf),
+ zkc,
bkc,
scheduler,
+ null,
NullStatsLogger.INSTANCE,
AsyncFailureInjector.NULL);
return new ReadAheadEntryReader(
@@ -309,7 +324,7 @@
BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
// generate list of log segments
- generateCompletedLogSegments(dlm, 3, 2);
+ generateCompletedLogSegments(dlm, 3, 3);
AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
FutureUtils.result(writer.truncate(new DLSN(2L, 1L, 0L)));
@@ -321,23 +336,39 @@
readAheadEntryReader.start(segments);
// ensure initialization to complete
ensureOrderSchedulerEmpty(streamName);
- expectAlreadyTruncatedTransactionException(readAheadEntryReader,
- "should fail on positioning to a truncated log segment");
+ expectNoException(readAheadEntryReader);
+ Entry.Reader entryReader =
+ readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+ assertEquals(2L, entryReader.getLSSN());
+ assertEquals(1L, entryReader.getEntryId());
+ Utils.close(readAheadEntryReader);
// positioning on a partially truncated log segment (segment 2) before min active dlsn
readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 0L, 0L), dlm, baseConf);
readAheadEntryReader.start(segments);
// ensure initialization to complete
ensureOrderSchedulerEmpty(streamName);
- expectAlreadyTruncatedTransactionException(readAheadEntryReader,
- "should fail on positioning to a partially truncated log segment");
+ expectNoException(readAheadEntryReader);
+ entryReader =
+ readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+ assertEquals(2L, entryReader.getLSSN());
+ assertEquals(1L, entryReader.getEntryId());
+ Utils.close(readAheadEntryReader);
// positioning on a partially truncated log segment (segment 2) after min active dlsn
- readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 1L, 0L), dlm, baseConf);
+ readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 2L, 0L), dlm, baseConf);
readAheadEntryReader.start(segments);
// ensure initialization to complete
ensureOrderSchedulerEmpty(streamName);
expectNoException(readAheadEntryReader);
+ entryReader =
+ readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+ assertEquals(2L, entryReader.getLSSN());
+ assertEquals(2L, entryReader.getEntryId());
+ Utils.close(readAheadEntryReader);
+
+ Utils.close(writer);
+ dlm.close();
}
@Test(timeout = 60000)
@@ -363,6 +394,7 @@
// ensure initialization to complete
ensureOrderSchedulerEmpty(streamName);
expectNoException(readAheadEntryReader);
+ Utils.close(readAheadEntryReader);
// positioning on a partially truncated log segment (segment 2) before min active dlsn
readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 0L, 0L), dlm, confLocal);
@@ -370,6 +402,7 @@
// ensure initialization to complete
ensureOrderSchedulerEmpty(streamName);
expectNoException(readAheadEntryReader);
+ Utils.close(readAheadEntryReader);
// positioning on a partially truncated log segment (segment 2) after min active dlsn
readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 1L, 0L), dlm, confLocal);
@@ -377,6 +410,10 @@
// ensure initialization to complete
ensureOrderSchedulerEmpty(streamName);
expectNoException(readAheadEntryReader);
+ Utils.close(readAheadEntryReader);
+
+ Utils.close(writer);
+ dlm.close();
}
//
@@ -418,6 +455,9 @@
ensureOrderSchedulerEmpty(streamName);
expectIllegalStateException(readAheadEntryReader,
"inconsistent log segment found");
+
+ Utils.close(readAheadEntryReader);
+ dlm.close();
}
}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java
index 99ef041..b183b84 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java
@@ -331,6 +331,7 @@
@FlakyTest
@Test(timeout = 60000)
+ @SuppressWarnings("deprecation")
public void testCaughtUpReaderOnLogSegmentRolling() throws Exception {
String name = "distrlog-caughtup-reader-on-logsegment-rolling";
@@ -344,6 +345,8 @@
confLocal.setWriteQuorumSize(1);
confLocal.setAckQuorumSize(1);
confLocal.setReadLACLongPollTimeout(99999999);
+ confLocal.setReaderIdleWarnThresholdMillis(2 * 99999999 + 1);
+ confLocal.setBKClientReadTimeout(99999999 + 1);
DistributedLogManager dlm = createNewDLM(confLocal, name);
BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
@@ -368,7 +371,7 @@
}
BKLogSegmentWriter perStreamWriter = writer.segmentWriter;
- BookKeeperClient bkc = readDLM.getReaderBKC();
+ BookKeeperClient bkc = DLMTestUtil.getBookKeeperClient(readDLM);
LedgerHandle readLh = bkc.get().openLedgerNoRecovery(getLedgerHandle(perStreamWriter).getId(),
BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java
index ee2968d..ff924f8 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java
@@ -20,6 +20,7 @@
import com.twitter.distributedlog.TestZooKeeperClientBuilder;
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.ZooKeeperClusterTestCase;
+import com.twitter.distributedlog.impl.acl.ZKAccessControl;
import com.twitter.distributedlog.thrift.AccessControlEntry;
import com.twitter.util.Await;
import org.apache.zookeeper.CreateMode;
@@ -103,7 +104,7 @@
ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
- assertEquals(zkPath, readZKAC.zkPath);
+ assertEquals(zkPath, readZKAC.getZKPath());
assertEquals(ZKAccessControl.DEFAULT_ACCESS_CONTROL_ENTRY, readZKAC.getAccessControlEntry());
assertTrue(ZKAccessControl.DEFAULT_ACCESS_CONTROL_ENTRY == readZKAC.getAccessControlEntry());
}
@@ -145,7 +146,7 @@
} catch (KeeperException.BadVersionException bve) {
// expected
}
- readZKAC2.accessControlEntry.setDenyTruncate(true);
+ readZKAC2.getAccessControlEntry().setDenyTruncate(true);
Await.result(readZKAC2.update(zkc));
ZKAccessControl readZKAC3 = Await.result(ZKAccessControl.read(zkc, zkPath, null));
assertEquals(readZKAC2, readZKAC3);
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java
index 8ba82f5..5625306 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java
@@ -22,6 +22,8 @@
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.ZooKeeperClientUtils;
import com.twitter.distributedlog.ZooKeeperClusterTestCase;
+import com.twitter.distributedlog.impl.acl.ZKAccessControl;
+import com.twitter.distributedlog.impl.acl.ZKAccessControlManager;
import com.twitter.distributedlog.thrift.AccessControlEntry;
import com.twitter.util.Await;
import org.junit.After;
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java
index d1069c3..60bc420 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java
@@ -28,6 +28,8 @@
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
+import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
import com.twitter.distributedlog.util.OrderedScheduler;
import com.twitter.distributedlog.util.SchedulerUtils;
import org.apache.zookeeper.CreateMode;
@@ -103,8 +105,10 @@
confLocal.setLogSegmentCacheEnabled(false);
URI uri = createDLMURI("/check-and-repair-dl-namespace");
zkc.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- com.twitter.distributedlog.DistributedLogManagerFactory factory =
- new com.twitter.distributedlog.DistributedLogManagerFactory(confLocal, uri);
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+ .conf(confLocal)
+ .uri(uri)
+ .build();
OrderedScheduler scheduler = OrderedScheduler.newBuilder()
.name("dlck-tool")
.corePoolSize(1)
@@ -114,17 +118,20 @@
String streamName = "check-and-repair-dl-namespace";
// Create completed log segments
- DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+ DistributedLogManager dlm = namespace.openLog(streamName);
DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 1L, 1L, 10, false);
DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 2L, 11L, 10, true);
DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 3L, 21L, 10, false);
DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 4L, 31L, 10, true);
// dryrun
- BookKeeperClient bkc = getBookKeeperClient(factory);
- DistributedLogAdmin.checkAndRepairDLNamespace(uri, factory,
- new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(factory)),
- scheduler, bkc, confLocal.getBKDigestPW(), false, false);
+ DistributedLogAdmin.checkAndRepairDLNamespace(
+ uri,
+ namespace,
+ new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(namespace)),
+ scheduler,
+ false,
+ false);
Map<Long, LogSegmentMetadata> segments = getLogSegments(dlm);
LOG.info("segments after drynrun {}", segments);
@@ -134,10 +141,13 @@
verifyLogSegment(segments, new DLSN(4L, 16L, 0L), 4L, 9, 39L);
// check and repair
- bkc = getBookKeeperClient(factory);
- DistributedLogAdmin.checkAndRepairDLNamespace(uri, factory,
- LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(factory)),
- scheduler, bkc, confLocal.getBKDigestPW(), false, false);
+ DistributedLogAdmin.checkAndRepairDLNamespace(
+ uri,
+ namespace,
+ LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(namespace)),
+ scheduler,
+ false,
+ false);
segments = getLogSegments(dlm);
LOG.info("segments after repair {}", segments);
@@ -148,7 +158,7 @@
dlm.close();
SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
- factory.close();
+ namespace.close();
}
}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java
index 66d7228..1e39e49 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java
@@ -24,6 +24,8 @@
import com.twitter.distributedlog.TestZooKeeperClientBuilder;
import com.twitter.distributedlog.annotations.DistributedLogAnnotations;
import com.twitter.distributedlog.exceptions.UnexpectedException;
+import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
import com.twitter.distributedlog.util.Utils;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.ZooDefs;
@@ -90,21 +92,25 @@
URI uri = createDLMURI("/change-sequence-number");
zooKeeperClient.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
- com.twitter.distributedlog.DistributedLogManagerFactory factory =
- new com.twitter.distributedlog.DistributedLogManagerFactory(confLocal, uri);
- com.twitter.distributedlog.DistributedLogManagerFactory readFactory =
- new com.twitter.distributedlog.DistributedLogManagerFactory(readConf, uri);
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+ .conf(confLocal)
+ .uri(uri)
+ .build();
+ DistributedLogNamespace readNamespace = DistributedLogNamespaceBuilder.newBuilder()
+ .conf(readConf)
+ .uri(uri)
+ .build();
String streamName = "change-sequence-number";
// create completed log segments
- DistributedLogManager dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+ DistributedLogManager dlm = namespace.openLog(streamName);
DLMTestUtil.generateCompletedLogSegments(dlm, confLocal, 4, 10);
DLMTestUtil.injectLogSegmentWithGivenLogSegmentSeqNo(dlm, confLocal, 5, 41, false, 10, true);
dlm.close();
// create a reader
- DistributedLogManager readDLM = readFactory.createDistributedLogManagerWithSharedClients(streamName);
+ DistributedLogManager readDLM = readNamespace.openLog(streamName);
AsyncLogReader reader = readDLM.getAsyncLogReader(DLSN.InitialDLSN);
// read the records
@@ -121,7 +127,7 @@
LOG.info("Injecting bad log segment '3'");
- dlm = factory.createDistributedLogManagerWithSharedClients(streamName);
+ dlm = namespace.openLog(streamName);
DLMTestUtil.injectLogSegmentWithGivenLogSegmentSeqNo(dlm, confLocal, 3L, 5 * 10 + 1, true, 10, false);
LOG.info("Injected bad log segment '3'");
@@ -140,8 +146,8 @@
LOG.info("Dryrun fix inprogress segment that has lower sequence number");
// Dryrun
- DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(factory,
- new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(factory)), streamName, false, false);
+ DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
+ new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
try {
reader = readDLM.getAsyncLogReader(lastDLSN);
@@ -154,8 +160,8 @@
LOG.info("Actual run fix inprogress segment that has lower sequence number");
// Actual run
- DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(factory,
- LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(factory)), streamName, false, false);
+ DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
+ LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
// be able to read more after fix
reader = readDLM.getAsyncLogReader(lastDLSN);
@@ -182,7 +188,7 @@
readDLM.close();
dlm.close();
- factory.close();
- readFactory.close();
+ namespace.close();
+ readNamespace.close();
}
}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java
index 46e8af0..de7016a 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java
@@ -531,7 +531,7 @@
children, firstSegmentList);
ZooKeeperClientUtils.expireSession(zkc,
- DLUtils.getZKServersFromDLUri(uri), conf.getZKSessionTimeoutMilliseconds());
+ BKNamespaceDriver.getZKServersFromDLUri(uri), conf.getZKSessionTimeoutMilliseconds());
logger.info("Create another {} segments.", numSegments);
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java
index da9f577..c9a2e5b 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java
@@ -174,7 +174,7 @@
createLogInNamespace(uri, "test2");
latches[2].await();
assertEquals(2, receivedLogs.get().size());
- ZooKeeperClientUtils.expireSession(zkc, DLUtils.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
+ ZooKeeperClientUtils.expireSession(zkc, BKNamespaceDriver.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
latches[3].await();
assertEquals(2, receivedLogs.get().size());
createLogInNamespace(uri, "test3");
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
index 673d856..0ce9f46 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
@@ -30,6 +30,7 @@
import com.twitter.distributedlog.callback.NamespaceListener;
import com.twitter.distributedlog.exceptions.LogExistsException;
import com.twitter.distributedlog.exceptions.UnexpectedException;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.metadata.LogMetadataStore;
import com.twitter.distributedlog.util.DLUtils;
import com.twitter.distributedlog.util.FutureUtils;
@@ -422,7 +423,7 @@
TestNamespaceListenerWithExpectedSize listener =
new TestNamespaceListenerWithExpectedSize(2 * maxLogsPerSubnamespace + 1);
metadataStore.registerNamespaceListener(listener);
- ZooKeeperClientUtils.expireSession(zkc, DLUtils.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
+ ZooKeeperClientUtils.expireSession(zkc, BKNamespaceDriver.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
String testLogName = "test-log-name";
allLogs.add(testLogName);
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
index 4cf86fa..183a405 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
@@ -30,10 +30,13 @@
import com.twitter.distributedlog.LogRecordWithDLSN;
import com.twitter.distributedlog.LogSegmentMetadata;
import com.twitter.distributedlog.TestDistributedLogBase;
+import com.twitter.distributedlog.ZooKeeperClient;
+import com.twitter.distributedlog.ZooKeeperClientBuilder;
import com.twitter.distributedlog.exceptions.EndOfLogSegmentException;
import com.twitter.distributedlog.exceptions.ReadCancelledException;
import com.twitter.distributedlog.injector.AsyncFailureInjector;
import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
+import com.twitter.distributedlog.util.ConfUtils;
import com.twitter.distributedlog.util.FutureUtils;
import com.twitter.distributedlog.util.OrderedScheduler;
import com.twitter.distributedlog.util.Utils;
@@ -59,10 +62,17 @@
public TestName runtime = new TestName();
private OrderedScheduler scheduler;
private BookKeeperClient bkc;
+ private ZooKeeperClient zkc;
@Before
public void setup() throws Exception {
super.setup();
+ zkc = ZooKeeperClientBuilder.newBuilder()
+ .name("test-zk")
+ .zkServers(bkutil.getZkServers())
+ .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+ .zkAclId(conf.getZkAclId())
+ .build();
bkc = BookKeeperClientBuilder.newBuilder()
.name("test-bk")
.dlConfig(conf)
@@ -83,6 +93,9 @@
if (null != scheduler) {
scheduler.shutdown();
}
+ if (null != zkc) {
+ zkc.close();
+ }
super.teardown();
}
@@ -91,7 +104,14 @@
DistributedLogConfiguration conf)
throws Exception {
LogSegmentEntryStore store = new BKLogSegmentEntryStore(
- conf, bkc, scheduler, NullStatsLogger.INSTANCE, AsyncFailureInjector.NULL);
+ conf,
+ ConfUtils.getConstDynConf(conf),
+ zkc,
+ bkc,
+ scheduler,
+ null,
+ NullStatsLogger.INSTANCE,
+ AsyncFailureInjector.NULL);
return (BKLogSegmentEntryReader) FutureUtils.result(store.openReader(segment, startEntryId));
}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
index 41544d6..1b19b2e 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
@@ -17,16 +17,16 @@
*/
package com.twitter.distributedlog.impl.metadata;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.DLMetadata;
import com.google.common.collect.Lists;
import com.twitter.distributedlog.DLMTestUtil;
import com.twitter.distributedlog.DistributedLogConfiguration;
+import com.twitter.distributedlog.MetadataAccessor;
+import com.twitter.distributedlog.TestZooKeeperClientBuilder;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
+import com.twitter.distributedlog.metadata.DLMetadata;
import com.twitter.distributedlog.metadata.LogMetadataForWriter;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.DistributedLogManager;
import com.twitter.distributedlog.DistributedLogConstants;
import com.twitter.distributedlog.exceptions.LogNotFoundException;
import com.twitter.distributedlog.ZooKeeperClient;
@@ -317,9 +317,9 @@
.uri(uri)
.build();
- DistributedLogManager dlm = namespace.openLog(logName);
- dlm.createOrUpdateMetadata(logName.getBytes("UTF-8"));
- dlm.close();
+ MetadataAccessor accessor = namespace.getNamespaceDriver().getMetadataAccessor(logName);
+ accessor.createOrUpdateMetadata(logName.getBytes("UTF-8"));
+ accessor.close();
testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, false);
}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestZkMetadataResolver.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZkMetadataResolver.java
similarity index 97%
rename from distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestZkMetadataResolver.java
rename to distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZkMetadataResolver.java
index 79fb539..bbabbb2 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestZkMetadataResolver.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZkMetadataResolver.java
@@ -15,11 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package com.twitter.distributedlog.metadata;
+package com.twitter.distributedlog.impl.metadata;
import com.twitter.distributedlog.DistributedLogConfiguration;
import com.twitter.distributedlog.DistributedLogConstants;
import com.twitter.distributedlog.TestZooKeeperClientBuilder;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.ZkMetadataResolver;
+import com.twitter.distributedlog.metadata.DLMetadata;
import com.twitter.distributedlog.util.Utils;
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.ZooKeeperClientBuilder;
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java
index d4c2f31..e3cc239 100644
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java
+++ b/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java
@@ -19,6 +19,7 @@
import com.twitter.distributedlog.LocalDLMEmulator;
import com.twitter.distributedlog.ZooKeeperClusterTestCase;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java
index 3225ced..a2a0ca6 100644
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java
+++ b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java
@@ -20,7 +20,7 @@
import com.twitter.distributedlog.DistributedLogConfiguration;
import com.twitter.distributedlog.LocalDLMEmulator;
import com.twitter.distributedlog.client.routing.SingleHostRoutingService;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import com.twitter.distributedlog.metadata.DLMetadata;
import com.twitter.distributedlog.service.placement.EqualLoadAppraiser;
import com.twitter.distributedlog.service.streamset.IdentityStreamPartitionConverter;
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java
index cfb5b8d..2e49d92 100644
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java
+++ b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java
@@ -23,12 +23,12 @@
import com.twitter.common.zookeeper.ServerSet;
import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
import com.twitter.distributedlog.client.serverset.DLZkServerSet;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.service.ClientUtils;
import com.twitter.distributedlog.service.DLSocketAddress;
import com.twitter.distributedlog.service.DistributedLogClient;
import com.twitter.distributedlog.service.DistributedLogClientBuilder;
import com.twitter.distributedlog.tools.Tool;
-import com.twitter.distributedlog.util.DLUtils;
import com.twitter.finagle.builder.ClientBuilder;
import com.twitter.finagle.thrift.ClientId$;
import com.twitter.util.Await;
@@ -269,8 +269,8 @@
ClientUtils.buildClient(builder2);
try {
SimpleBalancer balancer = new SimpleBalancer(
- DLUtils.getZKServersFromDLUri(region1), pair1.getLeft(), pair1.getRight(),
- DLUtils.getZKServersFromDLUri(region2), pair2.getLeft(), pair2.getRight());
+ BKNamespaceDriver.getZKServersFromDLUri(region1), pair1.getLeft(), pair1.getRight(),
+ BKNamespaceDriver.getZKServersFromDLUri(region2), pair2.getLeft(), pair2.getRight());
try {
return runBalancer(balancer);
} finally {
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java
index 18b9d1f..4f01bdc 100644
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java
+++ b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java
@@ -38,7 +38,7 @@
import com.twitter.distributedlog.BKDistributedLogNamespace;
import com.twitter.distributedlog.DistributedLogConfiguration;
import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.util.DLUtils;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
import com.twitter.distributedlog.util.Utils;
/**
@@ -55,11 +55,12 @@
private boolean watching = false;
public ZKPlacementStateManager(URI uri, DistributedLogConfiguration conf, StatsLogger statsLogger) {
- zkClient = BKDistributedLogNamespace.createDLZKClientBuilder(
- String.format("dlzk:%s:factory_writer_shared", uri),
+ String zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
+ zkClient = BKNamespaceDriver.createZKClientBuilder(
+ String.format("ZKPlacementStateManager-%s", zkServers),
conf,
- DLUtils.getZKServersFromDLUri(uri),
- statsLogger.scope("dlzk_factory_writer_shared")).build();
+ zkServers,
+ statsLogger.scope("placement_state_manager")).build();
serverLoadPath = uri.getPath() + SERVER_LOAD_DIR;
}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java
index c45e42c..218ea06 100644
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java
+++ b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java
@@ -30,10 +30,10 @@
import com.twitter.distributedlog.LogRecordWithDLSN;
import com.twitter.distributedlog.ZooKeeperClient;
import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.acl.ZKAccessControl;
+import com.twitter.distributedlog.impl.acl.ZKAccessControl;
import com.twitter.distributedlog.client.routing.LocalRoutingService;
import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.metadata.BKDLConfig;
+import com.twitter.distributedlog.impl.metadata.BKDLConfig;
import com.twitter.distributedlog.namespace.DistributedLogNamespace;
import com.twitter.distributedlog.service.stream.StreamManagerImpl;
import com.twitter.distributedlog.thrift.AccessControlEntry;
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java
index 20c81f3..87ddec0 100644
--- a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java
+++ b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java
@@ -18,12 +18,14 @@
package com.twitter.distributedlog.mapreduce;
import com.google.common.collect.Lists;
-import com.twitter.distributedlog.BKDistributedLogNamespace;
-import com.twitter.distributedlog.DLSN;
import com.twitter.distributedlog.DistributedLogConfiguration;
import com.twitter.distributedlog.DistributedLogManager;
+import com.twitter.distributedlog.DLSN;
import com.twitter.distributedlog.LogRecordWithDLSN;
import com.twitter.distributedlog.LogSegmentMetadata;
+import com.twitter.distributedlog.impl.BKNamespaceDriver;
+import com.twitter.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.BookKeeperAccessor;
@@ -57,7 +59,7 @@
protected Configuration conf;
protected DistributedLogConfiguration dlConf;
protected URI dlUri;
- protected BKDistributedLogNamespace namespace;
+ protected DistributedLogNamespace namespace;
protected String streamName;
protected DistributedLogManager dlm;
@@ -69,7 +71,7 @@
dlUri = URI.create(configuration.get(DL_URI, ""));
streamName = configuration.get(DL_STREAM, "");
try {
- namespace = BKDistributedLogNamespace.newBuilder()
+ namespace = DistributedLogNamespaceBuilder.newBuilder()
.conf(dlConf)
.uri(dlUri)
.build();
@@ -89,7 +91,7 @@
throws IOException, InterruptedException {
List<LogSegmentMetadata> segments = dlm.getLogSegments();
List<InputSplit> inputSplits = Lists.newArrayListWithCapacity(segments.size());
- BookKeeper bk = namespace.getReaderBKC().get();
+ BookKeeper bk = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get();
LedgerManager lm = BookKeeperAccessor.getLedgerManager(bk);
final AtomicInteger rcHolder = new AtomicInteger(0);
final AtomicReference<LedgerMetadata> metadataHolder = new AtomicReference<LedgerMetadata>(null);
@@ -121,7 +123,7 @@
return new LogSegmentReader(
streamName,
dlConf,
- namespace.getReaderBKC().get(),
+ ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get(),
(LogSegmentSplit) inputSplit);
}
}