You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@distributedlog.apache.org by si...@apache.org on 2017/01/05 00:51:06 UTC
[01/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Repository: incubator-distributedlog
Updated Branches:
refs/heads/master 7fab246d9 -> b44820b50
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
new file mode 100644
index 0000000..56a4f2e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
@@ -0,0 +1,630 @@
+/**
+ * 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 org.apache.distributedlog.auditor;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.SettableFuture;
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.BookKeeperClientBuilder;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.util.DLUtils;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAccessor;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * DL Auditor will audit DL namespace, e.g. find leaked ledger, report disk usage by streams.
+ */
+public class DLAuditor {
+
+ private static final Logger logger = LoggerFactory.getLogger(DLAuditor.class);
+
+ private final DistributedLogConfiguration conf;
+
+ public DLAuditor(DistributedLogConfiguration conf) {
+ this.conf = conf;
+ }
+
+ private ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) {
+ NamespaceDriver driver = namespace.getNamespaceDriver();
+ assert(driver instanceof BKNamespaceDriver);
+ return ((BKNamespaceDriver) driver).getWriterZKC();
+ }
+
+ 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 = BKNamespaceDriver.getZKServersFromDLUri(firstURI);
+ for (URI uri : uris) {
+ if (!zkServers.equalsIgnoreCase(BKNamespaceDriver.getZKServersFromDLUri(uri))) {
+ throw new IllegalArgumentException("Uris don't belong to same zookeeper cluster");
+ }
+ }
+ return zkServers;
+ }
+
+ private BKDLConfig resolveBKDLConfig(ZooKeeperClient zkc, List<URI> uris) throws IOException {
+ URI firstURI = uris.get(0);
+ BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, firstURI);
+ for (URI uri : uris) {
+ BKDLConfig anotherConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+ if (!(Objects.equal(bkdlConfig.getBkLedgersPath(), anotherConfig.getBkLedgersPath())
+ && Objects.equal(bkdlConfig.getBkZkServersForWriter(), anotherConfig.getBkZkServersForWriter()))) {
+ throw new IllegalArgumentException("Uris don't use same bookkeeper cluster");
+ }
+ }
+ return bkdlConfig;
+ }
+
+ public Pair<Set<Long>, Set<Long>> collectLedgers(List<URI> uris, List<List<String>> allocationPaths)
+ throws IOException {
+ Preconditions.checkArgument(uris.size() > 0, "No uri provided to audit");
+
+ String zkServers = validateAndGetZKServers(uris);
+ RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
+ conf.getZKRetryBackoffStartMillis(),
+ conf.getZKRetryBackoffMaxMillis(),
+ Integer.MAX_VALUE);
+ ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
+ .name("DLAuditor-ZK")
+ .zkServers(zkServers)
+ .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+ .retryPolicy(retryPolicy)
+ .zkAclId(conf.getZkAclId())
+ .build();
+ ExecutorService executorService = Executors.newCachedThreadPool();
+ try {
+ BKDLConfig bkdlConfig = resolveBKDLConfig(zkc, uris);
+ logger.info("Resolved bookkeeper config : {}", bkdlConfig);
+
+ BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
+ .name("DLAuditor-BK")
+ .dlConfig(conf)
+ .zkServers(bkdlConfig.getBkZkServersForWriter())
+ .ledgersPath(bkdlConfig.getBkLedgersPath())
+ .build();
+ try {
+ Set<Long> bkLedgers = collectLedgersFromBK(bkc, executorService);
+ Set<Long> dlLedgers = collectLedgersFromDL(uris, allocationPaths);
+ return Pair.of(bkLedgers, dlLedgers);
+ } finally {
+ bkc.close();
+ }
+ } finally {
+ zkc.close();
+ executorService.shutdown();
+ }
+ }
+
+ /**
+ * Find leak ledgers phase 1: collect ledgers set.
+ */
+ private Set<Long> collectLedgersFromBK(BookKeeperClient bkc,
+ final ExecutorService executorService)
+ throws IOException {
+ LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
+
+ final Set<Long> ledgers = new HashSet<Long>();
+ final SettableFuture<Void> doneFuture = SettableFuture.create();
+
+ BookkeeperInternalCallbacks.Processor<Long> collector =
+ new BookkeeperInternalCallbacks.Processor<Long>() {
+ @Override
+ public void process(Long lid,
+ final AsyncCallback.VoidCallback cb) {
+ synchronized (ledgers) {
+ ledgers.add(lid);
+ if (0 == ledgers.size() % 1000) {
+ logger.info("Collected {} ledgers", ledgers.size());
+ }
+ }
+ executorService.submit(new Runnable() {
+ @Override
+ public void run() {
+ cb.processResult(BKException.Code.OK, null, null);
+ }
+ });
+
+ }
+ };
+ AsyncCallback.VoidCallback finalCb = new AsyncCallback.VoidCallback() {
+ @Override
+ public void processResult(int rc, String path, Object ctx) {
+ if (BKException.Code.OK == rc) {
+ doneFuture.set(null);
+ } else {
+ doneFuture.setException(BKException.create(rc));
+ }
+ }
+ };
+ lm.asyncProcessLedgers(collector, finalCb, null, BKException.Code.OK,
+ BKException.Code.ZKException);
+ try {
+ doneFuture.get();
+ logger.info("Collected total {} ledgers", ledgers.size());
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new DLInterruptedException("Interrupted on collecting ledgers : ", e);
+ } catch (ExecutionException e) {
+ if (e.getCause() instanceof IOException) {
+ throw (IOException)(e.getCause());
+ } else {
+ throw new IOException("Failed to collect ledgers : ", e.getCause());
+ }
+ }
+ return ledgers;
+ }
+
+ /**
+ * Find leak ledgers phase 2: collect ledgers from uris.
+ */
+ private Set<Long> collectLedgersFromDL(List<URI> uris, List<List<String>> allocationPaths)
+ throws IOException {
+ final Set<Long> ledgers = new TreeSet<Long>();
+ List<DistributedLogNamespace> namespaces =
+ new ArrayList<DistributedLogNamespace>(uris.size());
+ try {
+ for (URI uri : uris) {
+ 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 (final DistributedLogNamespace namespace : namespaces) {
+ final DistributedLogNamespace dlNamespace = namespace;
+ final URI uri = uris.get(i);
+ final List<String> aps = allocationPaths.get(i);
+ i++;
+ executor.submit(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ logger.info("Collecting ledgers from {} : {}", uri, aps);
+ collectLedgersFromAllocator(uri, namespace, aps, ledgers);
+ synchronized (ledgers) {
+ logger.info("Collected {} ledgers from allocators for {} : {} ",
+ new Object[]{ledgers.size(), uri, ledgers});
+ }
+ collectLedgersFromDL(uri, namespace, ledgers);
+ } catch (IOException e) {
+ numFailures.incrementAndGet();
+ logger.info("Error to collect ledgers from DL : ", e);
+ }
+ doneLatch.countDown();
+ }
+ });
+ }
+ try {
+ doneLatch.await();
+ if (numFailures.get() > 0) {
+ throw new IOException(numFailures.get() + " errors to collect ledgers from DL");
+ }
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ logger.warn("Interrupted on collecting ledgers from DL : ", e);
+ throw new DLInterruptedException("Interrupted on collecting ledgers from DL : ", e);
+ }
+ } finally {
+ executor.shutdown();
+ }
+ } finally {
+ for (DistributedLogNamespace namespace : namespaces) {
+ namespace.close();
+ }
+ }
+ return ledgers;
+ }
+
+ private void collectLedgersFromAllocator(final URI uri,
+ final DistributedLogNamespace namespace,
+ final List<String> allocationPaths,
+ final Set<Long> ledgers) throws IOException {
+ final LinkedBlockingQueue<String> poolQueue =
+ new LinkedBlockingQueue<String>();
+ for (String allocationPath : allocationPaths) {
+ String rootPath = uri.getPath() + "/" + allocationPath;
+ try {
+ List<String> pools = getZooKeeperClient(namespace).get().getChildren(rootPath, false);
+ for (String pool : pools) {
+ poolQueue.add(rootPath + "/" + pool);
+ }
+ } catch (KeeperException e) {
+ throw new ZKException("Failed to get list of pools from " + rootPath, e);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new DLInterruptedException("Interrupted on getting list of pools from " + rootPath, e);
+ }
+ }
+
+
+ logger.info("Collecting ledgers from allocators for {} : {}", uri, poolQueue);
+
+ executeAction(poolQueue, 10, new Action<String>() {
+ @Override
+ public void execute(String poolPath) throws IOException {
+ try {
+ collectLedgersFromPool(poolPath);
+ } catch (InterruptedException e) {
+ throw new DLInterruptedException("Interrupted on collecting ledgers from allocation pool " + poolPath, e);
+ } catch (KeeperException e) {
+ throw new ZKException("Failed to collect ledgers from allocation pool " + poolPath, e.code());
+ }
+ }
+
+ private void collectLedgersFromPool(String poolPath)
+ throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
+ List<String> allocators = getZooKeeperClient(namespace).get()
+ .getChildren(poolPath, false);
+ for (String allocator : allocators) {
+ String allocatorPath = poolPath + "/" + allocator;
+ byte[] data = getZooKeeperClient(namespace).get().getData(allocatorPath, false, new Stat());
+ if (null != data && data.length > 0) {
+ try {
+ long ledgerId = DLUtils.bytes2LogSegmentId(data);
+ synchronized (ledgers) {
+ ledgers.add(ledgerId);
+ }
+ } catch (NumberFormatException nfe) {
+ logger.warn("Invalid ledger found in allocator path {} : ", allocatorPath, nfe);
+ }
+ }
+ }
+ }
+ });
+
+ logger.info("Collected ledgers from allocators for {}.", uri);
+ }
+
+ private void collectLedgersFromDL(final URI uri,
+ final DistributedLogNamespace namespace,
+ final Set<Long> ledgers) throws IOException {
+ logger.info("Enumerating {} to collect streams.", uri);
+ Iterator<String> streams = namespace.getLogs();
+ final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
+ while (streams.hasNext()) {
+ streamQueue.add(streams.next());
+ }
+
+ logger.info("Collected {} streams from uri {} : {}",
+ new Object[] { streamQueue.size(), uri, streams });
+
+ executeAction(streamQueue, 10, new Action<String>() {
+ @Override
+ public void execute(String stream) throws IOException {
+ collectLedgersFromStream(namespace, stream, ledgers);
+ }
+ });
+ }
+
+ private List<Long> collectLedgersFromStream(DistributedLogNamespace namespace,
+ String stream,
+ Set<Long> ledgers)
+ throws IOException {
+ DistributedLogManager dlm = namespace.openLog(stream);
+ try {
+ List<LogSegmentMetadata> segments = dlm.getLogSegments();
+ List<Long> sLedgers = new ArrayList<Long>();
+ for (LogSegmentMetadata segment : segments) {
+ synchronized (ledgers) {
+ ledgers.add(segment.getLogSegmentId());
+ }
+ sLedgers.add(segment.getLogSegmentId());
+ }
+ return sLedgers;
+ } finally {
+ dlm.close();
+ }
+ }
+
+ /**
+ * Calculating stream space usage from given <i>uri</i>.
+ *
+ * @param uri dl uri
+ * @throws IOException
+ */
+ public Map<String, Long> calculateStreamSpaceUsage(final URI uri) throws IOException {
+ logger.info("Collecting stream space usage for {}.", uri);
+ DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+ .conf(conf)
+ .uri(uri)
+ .build();
+ try {
+ return calculateStreamSpaceUsage(uri, namespace);
+ } finally {
+ namespace.close();
+ }
+ }
+
+ private Map<String, Long> calculateStreamSpaceUsage(
+ final URI uri, final DistributedLogNamespace namespace)
+ throws IOException {
+ Iterator<String> streams = namespace.getLogs();
+ final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
+ while (streams.hasNext()) {
+ streamQueue.add(streams.next());
+ }
+
+ final Map<String, Long> streamSpaceUsageMap =
+ new ConcurrentSkipListMap<String, Long>();
+ final AtomicInteger numStreamsCollected = new AtomicInteger(0);
+
+ executeAction(streamQueue, 10, new Action<String>() {
+ @Override
+ public void execute(String stream) throws IOException {
+ streamSpaceUsageMap.put(stream,
+ calculateStreamSpaceUsage(namespace, stream));
+ if (numStreamsCollected.incrementAndGet() % 1000 == 0) {
+ logger.info("Calculated {} streams from uri {}.", numStreamsCollected.get(), uri);
+ }
+ }
+ });
+
+ return streamSpaceUsageMap;
+ }
+
+ private long calculateStreamSpaceUsage(final DistributedLogNamespace namespace,
+ final String stream) throws IOException {
+ DistributedLogManager dlm = namespace.openLog(stream);
+ long totalBytes = 0;
+ try {
+ List<LogSegmentMetadata> segments = dlm.getLogSegments();
+ for (LogSegmentMetadata segment : segments) {
+ try {
+ LedgerHandle lh = getBookKeeperClient(namespace).get().openLedgerNoRecovery(segment.getLogSegmentId(),
+ BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
+ totalBytes += lh.getLength();
+ lh.close();
+ } catch (BKException e) {
+ logger.error("Failed to open ledger {} : ", segment.getLogSegmentId(), e);
+ throw new IOException("Failed to open ledger " + segment.getLogSegmentId(), e);
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted on opening ledger {} : ", segment.getLogSegmentId(), e);
+ Thread.currentThread().interrupt();
+ throw new DLInterruptedException("Interrupted on opening ledger " + segment.getLogSegmentId(), e);
+ }
+ }
+ } finally {
+ dlm.close();
+ }
+ return totalBytes;
+ }
+
+ public long calculateLedgerSpaceUsage(URI uri) throws IOException {
+ List<URI> uris = Lists.newArrayList(uri);
+ String zkServers = validateAndGetZKServers(uris);
+ RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
+ conf.getZKRetryBackoffStartMillis(),
+ conf.getZKRetryBackoffMaxMillis(),
+ Integer.MAX_VALUE);
+ ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
+ .name("DLAuditor-ZK")
+ .zkServers(zkServers)
+ .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+ .retryPolicy(retryPolicy)
+ .zkAclId(conf.getZkAclId())
+ .build();
+ ExecutorService executorService = Executors.newCachedThreadPool();
+ try {
+ BKDLConfig bkdlConfig = resolveBKDLConfig(zkc, uris);
+ logger.info("Resolved bookkeeper config : {}", bkdlConfig);
+
+ BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
+ .name("DLAuditor-BK")
+ .dlConfig(conf)
+ .zkServers(bkdlConfig.getBkZkServersForWriter())
+ .ledgersPath(bkdlConfig.getBkLedgersPath())
+ .build();
+ try {
+ return calculateLedgerSpaceUsage(bkc, executorService);
+ } finally {
+ bkc.close();
+ }
+ } finally {
+ zkc.close();
+ executorService.shutdown();
+ }
+ }
+
+ private long calculateLedgerSpaceUsage(BookKeeperClient bkc,
+ final ExecutorService executorService)
+ throws IOException {
+ final AtomicLong totalBytes = new AtomicLong(0);
+ final AtomicLong totalEntries = new AtomicLong(0);
+ final AtomicLong numLedgers = new AtomicLong(0);
+
+ LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
+
+ final SettableFuture<Void> doneFuture = SettableFuture.create();
+ final BookKeeper bk = bkc.get();
+
+ BookkeeperInternalCallbacks.Processor<Long> collector =
+ new BookkeeperInternalCallbacks.Processor<Long>() {
+ @Override
+ public void process(final Long lid,
+ final AsyncCallback.VoidCallback cb) {
+ numLedgers.incrementAndGet();
+ executorService.submit(new Runnable() {
+ @Override
+ public void run() {
+ bk.asyncOpenLedgerNoRecovery(lid, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8),
+ new org.apache.bookkeeper.client.AsyncCallback.OpenCallback() {
+ @Override
+ public void openComplete(int rc, LedgerHandle lh, Object ctx) {
+ final int cbRc;
+ if (BKException.Code.OK == rc) {
+ totalBytes.addAndGet(lh.getLength());
+ totalEntries.addAndGet(lh.getLastAddConfirmed() + 1);
+ cbRc = rc;
+ } else {
+ cbRc = BKException.Code.ZKException;
+ }
+ executorService.submit(new Runnable() {
+ @Override
+ public void run() {
+ cb.processResult(cbRc, null, null);
+ }
+ });
+ }
+ }, null);
+ }
+ });
+ }
+ };
+ AsyncCallback.VoidCallback finalCb = new AsyncCallback.VoidCallback() {
+ @Override
+ public void processResult(int rc, String path, Object ctx) {
+ if (BKException.Code.OK == rc) {
+ doneFuture.set(null);
+ } else {
+ doneFuture.setException(BKException.create(rc));
+ }
+ }
+ };
+ lm.asyncProcessLedgers(collector, finalCb, null, BKException.Code.OK, BKException.Code.ZKException);
+ try {
+ doneFuture.get();
+ logger.info("calculated {} ledgers\n\ttotal bytes = {}\n\ttotal entries = {}",
+ new Object[] { numLedgers.get(), totalBytes.get(), totalEntries.get() });
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new DLInterruptedException("Interrupted on calculating ledger space : ", e);
+ } catch (ExecutionException e) {
+ if (e.getCause() instanceof IOException) {
+ throw (IOException)(e.getCause());
+ } else {
+ throw new IOException("Failed to calculate ledger space : ", e.getCause());
+ }
+ }
+ return totalBytes.get();
+ }
+
+ public void close() {
+ // no-op
+ }
+
+ static interface Action<T> {
+ void execute(T item) throws IOException ;
+ }
+
+ static <T> void executeAction(final LinkedBlockingQueue<T> queue,
+ final int numThreads,
+ final Action<T> action) throws IOException {
+ final CountDownLatch failureLatch = new CountDownLatch(1);
+ final CountDownLatch doneLatch = new CountDownLatch(queue.size());
+ final AtomicInteger numFailures = new AtomicInteger(0);
+ final AtomicInteger completedThreads = new AtomicInteger(0);
+
+ ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+ try {
+ for (int i = 0 ; i < numThreads; i++) {
+ executorService.submit(new Runnable() {
+ @Override
+ public void run() {
+ while (true) {
+ T item = queue.poll();
+ if (null == item) {
+ break;
+ }
+ try {
+ action.execute(item);
+ } catch (IOException ioe) {
+ logger.error("Failed to execute action on item '{}'", item, ioe);
+ numFailures.incrementAndGet();
+ failureLatch.countDown();
+ break;
+ }
+ doneLatch.countDown();
+ }
+ if (numFailures.get() == 0 && completedThreads.incrementAndGet() == numThreads) {
+ failureLatch.countDown();
+ }
+ }
+ });
+ }
+ try {
+ failureLatch.await();
+ if (numFailures.get() > 0) {
+ throw new IOException("Encountered " + numFailures.get() + " failures on executing action.");
+ }
+ doneLatch.await();
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ logger.warn("Interrupted on executing action", ie);
+ throw new DLInterruptedException("Interrupted on executing action", ie);
+ }
+ } finally {
+ executorService.shutdown();
+ }
+ }
+
+}
[05/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
new file mode 100644
index 0000000..f951991
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
@@ -0,0 +1,69 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.ACL;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+public class DistributedLogConstants {
+ public static final byte[] EMPTY_BYTES = new byte[0];
+ public static final String SCHEME_PREFIX = "distributedlog";
+ public static final String BACKEND_BK = "bk";
+ public static final long INVALID_TXID = -999;
+ public static final long EMPTY_LOGSEGMENT_TX_ID = -99;
+ public static final long MAX_TXID = Long.MAX_VALUE;
+ public static final long SMALL_LOGSEGMENT_THRESHOLD = 10;
+ public static final int LOGSEGMENT_NAME_VERSION = 1;
+ public static final int FUTURE_TIMEOUT_IMMEDIATE = 0;
+ public static final int FUTURE_TIMEOUT_INFINITE = -1;
+ public static final long LOCK_IMMEDIATE = FUTURE_TIMEOUT_IMMEDIATE;
+ public static final long LOCK_TIMEOUT_INFINITE = FUTURE_TIMEOUT_INFINITE;
+ public static final long LOCK_OP_TIMEOUT_DEFAULT = 120;
+ public static final long LOCK_REACQUIRE_TIMEOUT_DEFAULT = 120;
+ public static final String UNKNOWN_CLIENT_ID = "Unknown-ClientId";
+ public static final int LOCAL_REGION_ID = 0;
+ public static final long LOGSEGMENT_DEFAULT_STATUS = 0;
+ public static final long UNASSIGNED_LOGSEGMENT_SEQNO = 0;
+ public static final long UNASSIGNED_SEQUENCE_ID = -1L;
+ public static final long FIRST_LOGSEGMENT_SEQNO = 1;
+ public static final long UNRESOLVED_LEDGER_ID = -1;
+ public static final long LATENCY_WARN_THRESHOLD_IN_MILLIS = TimeUnit.SECONDS.toMillis(1);
+ public static final int DL_INTERRUPTED_EXCEPTION_RESULT_CODE = Integer.MIN_VALUE + 1;
+ public static final int ZK_CONNECTION_EXCEPTION_RESULT_CODE = Integer.MIN_VALUE + 2;
+
+ public static final String ALLOCATION_POOL_NODE = ".allocation_pool";
+ // log segment prefix
+ public static final String INPROGRESS_LOGSEGMENT_PREFIX = "inprogress";
+ public static final String COMPLETED_LOGSEGMENT_PREFIX = "logrecs";
+ public static final String DISALLOW_PLACEMENT_IN_REGION_FEATURE_NAME = "disallow_bookie_placement";
+ static final byte[] CONTROL_RECORD_CONTENT = "control".getBytes(UTF_8);
+ static final byte[] KEEPALIVE_RECORD_CONTENT = "keepalive".getBytes(UTF_8);
+
+ // An ACL that gives all permissions to node creators and read permissions only to everyone else.
+ public static final List<ACL> EVERYONE_READ_CREATOR_ALL =
+ ImmutableList.<ACL>builder()
+ .addAll(Ids.CREATOR_ALL_ACL)
+ .addAll(Ids.READ_ACL_UNSAFE)
+ .build();
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
new file mode 100644
index 0000000..7d33e9c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
@@ -0,0 +1,308 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.subscription.SubscriptionStateStore;
+import org.apache.distributedlog.subscription.SubscriptionsStore;
+import com.twitter.util.Future;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * A DistributedLogManager is responsible for managing a single place of storing
+ * edit logs. It may correspond to multiple files, a backup node, etc.
+ * Even when the actual underlying storage is rolled, or failed and restored,
+ * 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 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.
+ *
+ * @return log segments
+ * @throws IOException
+ */
+ public List<LogSegmentMetadata> getLogSegments() throws IOException;
+
+ /**
+ * Register <i>listener</i> on log segment updates of this stream.
+ *
+ * @param listener
+ * listener to receive update log segment list.
+ */
+ public void registerListener(LogSegmentListener listener) throws IOException ;
+
+ /**
+ * Unregister <i>listener</i> on log segment updates from this stream.
+ *
+ * @param listener
+ * listener to receive update log segment list.
+ */
+ public void unregisterListener(LogSegmentListener listener);
+
+ /**
+ * Open async log writer to write records to the log stream.
+ *
+ * @return result represents the open result
+ */
+ public Future<AsyncLogWriter> openAsyncLogWriter();
+
+ /**
+ * Begin writing to the log stream identified by the name
+ *
+ * @return the writer interface to generate log records
+ */
+ public LogWriter startLogSegmentNonPartitioned() throws IOException;
+
+ /**
+ * Begin writing to the log stream identified by the name
+ *
+ * @return the writer interface to generate log records
+ */
+ // @Deprecated
+ public AsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException;
+
+ /**
+ * Begin appending to the end of the log stream which is being treated as a sequence of bytes
+ *
+ * @return the writer interface to generate log records
+ */
+ public AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException;
+
+ /**
+ * Get a reader to read a log stream as a sequence of bytes
+ *
+ * @return the writer interface to generate log records
+ */
+ public AppendOnlyStreamReader getAppendOnlyStreamReader() throws IOException;
+
+ /**
+ * Get the input stream starting with fromTxnId for the specified log
+ *
+ * @param fromTxnId - the first transaction id we want to read
+ * @return the stream starting with transaction fromTxnId
+ * @throws IOException if a stream cannot be found.
+ */
+ public LogReader getInputStream(long fromTxnId)
+ throws IOException;
+
+ public LogReader getInputStream(DLSN fromDLSN) throws IOException;
+
+ /**
+ * Open an async log reader to read records from a log starting from <code>fromTxnId</code>.
+ *
+ * @param fromTxnId
+ * transaction id to start reading from
+ * @return async log reader
+ */
+ public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId);
+
+ /**
+ * Open an async log reader to read records from a log starting from <code>fromDLSN</code>
+ *
+ * @param fromDLSN
+ * dlsn to start reading from
+ * @return async log reader
+ */
+ public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN);
+
+ // @Deprecated
+ public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException;
+
+ // @Deprecated
+ public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException;
+
+ public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
+
+ /**
+ * Get a log reader with lock starting from <i>fromDLSN</i> and using <i>subscriberId</i>.
+ * If two readers tried to open using same subscriberId, one would succeed, while the other
+ * will be blocked until it gets the lock.
+ *
+ * @param fromDLSN
+ * start dlsn
+ * @param subscriberId
+ * subscriber id
+ * @return async log reader
+ */
+ public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN, String subscriberId);
+
+ /**
+ * Get a log reader using <i>subscriberId</i> with lock. The reader will start reading from
+ * its last commit position recorded in subscription store. If no last commit position found
+ * in subscription store, it would start reading from head of the stream.
+ *
+ * If the two readers tried to open using same subscriberId, one would succeed, while the other
+ * will be blocked until it gets the lock.
+ *
+ * @param subscriberId
+ * subscriber id
+ * @return async log reader
+ */
+ public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
+
+ /**
+ * Get the {@link DLSN} of first log record whose transaction id is not less than <code>transactionId</code>.
+ *
+ * @param transactionId
+ * transaction id
+ * @return dlsn of first log record whose transaction id is not less than transactionId.
+ */
+ public Future<DLSN> getDLSNNotLessThanTxId(long transactionId);
+
+ /**
+ * Get the last log record in the stream
+ *
+ * @return the last log record in the stream
+ * @throws IOException if a stream cannot be found.
+ */
+ public LogRecordWithDLSN getLastLogRecord()
+ throws IOException;
+
+ /**
+ * Get the earliest Transaction Id available in the log
+ *
+ * @return earliest transaction id
+ * @throws IOException
+ */
+ public long getFirstTxId() throws IOException;
+
+ /**
+ * Get Latest Transaction Id in the log
+ *
+ * @return latest transaction id
+ * @throws IOException
+ */
+ public long getLastTxId() throws IOException;
+
+ /**
+ * Get Latest DLSN in the log
+ *
+ * @return last dlsn
+ * @throws IOException
+ */
+ public DLSN getLastDLSN() throws IOException;
+
+ /**
+ * Get Latest log record with DLSN in the log - async
+ *
+ * @return latest log record with DLSN
+ */
+ public Future<LogRecordWithDLSN> getLastLogRecordAsync();
+
+ /**
+ * Get Latest Transaction Id in the log - async
+ *
+ * @return latest transaction id
+ */
+ public Future<Long> getLastTxIdAsync();
+
+ /**
+ * Get first DLSN in the log.
+ *
+ * @return first dlsn in the stream
+ */
+ public Future<DLSN> getFirstDLSNAsync();
+
+ /**
+ * Get Latest DLSN in the log - async
+ *
+ * @return latest transaction id
+ */
+ public Future<DLSN> getLastDLSNAsync();
+
+ /**
+ * Get the number of log records in the active portion of the log
+ * Any log segments that have already been truncated will not be included
+ *
+ * @return number of log records
+ * @throws IOException
+ */
+ public long getLogRecordCount() throws IOException;
+
+ /**
+ * Get the number of log records in the active portion of the log - async.
+ * Any log segments that have already been truncated will not be included
+ *
+ * @return future number of log records
+ * @throws IOException
+ */
+ public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN);
+
+ /**
+ * Run recovery on the log.
+ *
+ * @throws IOException
+ */
+ public void recover() throws IOException;
+
+ /**
+ * Check if an end of stream marker was added to the stream
+ * A stream with an end of stream marker cannot be appended to
+ *
+ * @return true if the marker was added to the stream, false otherwise
+ * @throws IOException
+ */
+ public boolean isEndOfStreamMarked() throws IOException;
+
+ /**
+ * Delete the log.
+ *
+ * @throws IOException if the deletion fails
+ */
+ public void delete() throws IOException;
+
+ /**
+ * The DistributedLogManager may archive/purge any logs for transactionId
+ * less than or equal to minImageTxId.
+ * This is to be used only when the client explicitly manages deletion. If
+ * the cleanup policy is based on sliding time window, then this method need
+ * not be called.
+ *
+ * @param minTxIdToKeep the earliest txid that must be retained
+ * @throws IOException if purging fails
+ */
+ public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException;
+
+ /**
+ * Get the subscriptions store provided by the distributedlog manager.
+ *
+ * @return subscriptions store manages subscriptions for current stream.
+ */
+ public SubscriptionsStore getSubscriptionsStore();
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java b/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
new file mode 100644
index 0000000..617282c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
@@ -0,0 +1,403 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.io.CompressionCodec;
+import com.twitter.util.Promise;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+import javax.annotation.Nullable;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A set of {@link LogRecord}s.
+ */
+public class Entry {
+
+ /**
+ * Create a new log record set.
+ *
+ * @param logName
+ * name of the log
+ * @param initialBufferSize
+ * initial buffer size
+ * @param envelopeBeforeTransmit
+ * if envelope the buffer before transmit
+ * @param codec
+ * compression codec
+ * @param statsLogger
+ * stats logger to receive stats
+ * @return writer to build a log record set.
+ */
+ public static Writer newEntry(
+ String logName,
+ int initialBufferSize,
+ boolean envelopeBeforeTransmit,
+ CompressionCodec.Type codec,
+ StatsLogger statsLogger) {
+ return new EnvelopedEntryWriter(
+ logName,
+ initialBufferSize,
+ envelopeBeforeTransmit,
+ codec,
+ statsLogger);
+ }
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ /**
+ * Build the record set object.
+ */
+ public static class Builder {
+
+ private long logSegmentSequenceNumber = -1;
+ private long entryId = -1;
+ private long startSequenceId = Long.MIN_VALUE;
+ private boolean envelopeEntry = true;
+ // input stream
+ private InputStream in = null;
+ // or bytes array
+ private byte[] data = null;
+ private int offset = -1;
+ private int length = -1;
+ private Optional<Long> txidToSkipTo = Optional.absent();
+ private Optional<DLSN> dlsnToSkipTo = Optional.absent();
+ private boolean deserializeRecordSet = true;
+
+ private Builder() {}
+
+ /**
+ * Reset the builder.
+ *
+ * @return builder
+ */
+ public Builder reset() {
+ logSegmentSequenceNumber = -1;
+ entryId = -1;
+ startSequenceId = Long.MIN_VALUE;
+ envelopeEntry = true;
+ // input stream
+ in = null;
+ // or bytes array
+ data = null;
+ offset = -1;
+ length = -1;
+ txidToSkipTo = Optional.absent();
+ dlsnToSkipTo = Optional.absent();
+ return this;
+ }
+
+ /**
+ * Set the segment info of the log segment that this record
+ * set belongs to.
+ *
+ * @param lssn
+ * log segment sequence number
+ * @param startSequenceId
+ * start sequence id of this log segment
+ * @return builder
+ */
+ public Builder setLogSegmentInfo(long lssn, long startSequenceId) {
+ this.logSegmentSequenceNumber = lssn;
+ this.startSequenceId = startSequenceId;
+ return this;
+ }
+
+ /**
+ * Set the entry id of this log record set.
+ *
+ * @param entryId
+ * entry id assigned for this log record set.
+ * @return builder
+ */
+ public Builder setEntryId(long entryId) {
+ this.entryId = entryId;
+ return this;
+ }
+
+ /**
+ * Set whether this record set is enveloped or not.
+ *
+ * @param enabled
+ * flag indicates whether this record set is enveloped or not.
+ * @return builder
+ */
+ public Builder setEnvelopeEntry(boolean enabled) {
+ this.envelopeEntry = enabled;
+ return this;
+ }
+
+ /**
+ * Set the serialized bytes data of this record set.
+ *
+ * @param data
+ * serialized bytes data of this record set.
+ * @param offset
+ * offset of the bytes data
+ * @param length
+ * length of the bytes data
+ * @return builder
+ */
+ public Builder setData(byte[] data, int offset, int length) {
+ this.data = data;
+ this.offset = offset;
+ this.length = length;
+ return this;
+ }
+
+ /**
+ * Set the input stream of the serialized bytes data of this record set.
+ *
+ * @param in
+ * input stream
+ * @return builder
+ */
+ public Builder setInputStream(InputStream in) {
+ this.in = in;
+ return this;
+ }
+
+ /**
+ * Set the record set starts from <code>dlsn</code>.
+ *
+ * @param dlsn
+ * dlsn to skip to
+ * @return builder
+ */
+ public Builder skipTo(@Nullable DLSN dlsn) {
+ this.dlsnToSkipTo = Optional.fromNullable(dlsn);
+ return this;
+ }
+
+ /**
+ * Set the record set starts from <code>txid</code>.
+ *
+ * @param txid
+ * txid to skip to
+ * @return builder
+ */
+ public Builder skipTo(long txid) {
+ this.txidToSkipTo = Optional.of(txid);
+ return this;
+ }
+
+ /**
+ * Enable/disable deserialize record set.
+ *
+ * @param enabled
+ * flag to enable/disable dserialize record set.
+ * @return builder
+ */
+ public Builder deserializeRecordSet(boolean enabled) {
+ this.deserializeRecordSet = enabled;
+ return this;
+ }
+
+ public Entry build() {
+ Preconditions.checkNotNull(data, "Serialized data isn't provided");
+ Preconditions.checkArgument(offset >= 0 && length >= 0
+ && (offset + length) <= data.length,
+ "Invalid offset or length of serialized data");
+ return new Entry(
+ logSegmentSequenceNumber,
+ entryId,
+ startSequenceId,
+ envelopeEntry,
+ deserializeRecordSet,
+ data,
+ offset,
+ length,
+ txidToSkipTo,
+ dlsnToSkipTo);
+ }
+
+ public Entry.Reader buildReader() throws IOException {
+ Preconditions.checkArgument(data != null || in != null,
+ "Serialized data or input stream isn't provided");
+ InputStream in;
+ if (null != this.in) {
+ in = this.in;
+ } else {
+ Preconditions.checkArgument(offset >= 0 && length >= 0
+ && (offset + length) <= data.length,
+ "Invalid offset or length of serialized data");
+ in = new ByteArrayInputStream(data, offset, length);
+ }
+ return new EnvelopedEntryReader(
+ logSegmentSequenceNumber,
+ entryId,
+ startSequenceId,
+ in,
+ envelopeEntry,
+ deserializeRecordSet,
+ NullStatsLogger.INSTANCE);
+ }
+
+ }
+
+ private final long logSegmentSequenceNumber;
+ private final long entryId;
+ private final long startSequenceId;
+ private final boolean envelopedEntry;
+ private final boolean deserializeRecordSet;
+ private final byte[] data;
+ private final int offset;
+ private final int length;
+ private final Optional<Long> txidToSkipTo;
+ private final Optional<DLSN> dlsnToSkipTo;
+
+ private Entry(long logSegmentSequenceNumber,
+ long entryId,
+ long startSequenceId,
+ boolean envelopedEntry,
+ boolean deserializeRecordSet,
+ byte[] data,
+ int offset,
+ int length,
+ Optional<Long> txidToSkipTo,
+ Optional<DLSN> dlsnToSkipTo) {
+ this.logSegmentSequenceNumber = logSegmentSequenceNumber;
+ this.entryId = entryId;
+ this.startSequenceId = startSequenceId;
+ this.envelopedEntry = envelopedEntry;
+ this.deserializeRecordSet = deserializeRecordSet;
+ this.data = data;
+ this.offset = offset;
+ this.length = length;
+ this.txidToSkipTo = txidToSkipTo;
+ this.dlsnToSkipTo = dlsnToSkipTo;
+ }
+
+ /**
+ * Get raw data of this record set.
+ *
+ * @return raw data representation of this record set.
+ */
+ public byte[] getRawData() {
+ return data;
+ }
+
+ /**
+ * Create reader to iterate over this record set.
+ *
+ * @return reader to iterate over this record set.
+ * @throws IOException if the record set is invalid record set.
+ */
+ public Reader reader() throws IOException {
+ InputStream in = new ByteArrayInputStream(data, offset, length);
+ Reader reader = new EnvelopedEntryReader(
+ logSegmentSequenceNumber,
+ entryId,
+ startSequenceId,
+ in,
+ envelopedEntry,
+ deserializeRecordSet,
+ NullStatsLogger.INSTANCE);
+ if (txidToSkipTo.isPresent()) {
+ reader.skipTo(txidToSkipTo.get());
+ }
+ if (dlsnToSkipTo.isPresent()) {
+ reader.skipTo(dlsnToSkipTo.get());
+ }
+ return reader;
+ }
+
+ /**
+ * Writer to append {@link LogRecord}s to {@link Entry}.
+ */
+ public interface Writer extends EntryBuffer {
+
+ /**
+ * Write a {@link LogRecord} to this record set.
+ *
+ * @param record
+ * record to write
+ * @param transmitPromise
+ * callback for transmit result. the promise is only
+ * satisfied when this record set is transmitted.
+ * @throws LogRecordTooLongException if the record is too long
+ * @throws WriteException when encountered exception writing the record
+ */
+ void writeRecord(LogRecord record, Promise<DLSN> transmitPromise)
+ throws LogRecordTooLongException, WriteException;
+
+ /**
+ * Reset the writer to write records.
+ */
+ void reset();
+
+ }
+
+ /**
+ * Reader to read {@link LogRecord}s from this record set.
+ */
+ public interface Reader {
+
+ /**
+ * Get the log segment sequence number.
+ *
+ * @return the log segment sequence number.
+ */
+ long getLSSN();
+
+ /**
+ * Return the entry id.
+ *
+ * @return the entry id.
+ */
+ long getEntryId();
+
+ /**
+ * Read next log record from this record set.
+ *
+ * @return next log record from this record set.
+ */
+ LogRecordWithDLSN nextRecord() throws IOException;
+
+ /**
+ * Skip the reader to the record whose transaction id is <code>txId</code>.
+ *
+ * @param txId
+ * transaction id to skip to.
+ * @return true if skip succeeds, otherwise false.
+ * @throws IOException
+ */
+ boolean skipTo(long txId) throws IOException;
+
+ /**
+ * Skip the reader to the record whose DLSN is <code>dlsn</code>.
+ *
+ * @param dlsn
+ * DLSN to skip to.
+ * @return true if skip succeeds, otherwise false.
+ * @throws IOException
+ */
+ boolean skipTo(DLSN dlsn) throws IOException;
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EntryBuffer.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EntryBuffer.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EntryBuffer.java
new file mode 100644
index 0000000..c695420
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EntryBuffer.java
@@ -0,0 +1,70 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
+import org.apache.distributedlog.io.Buffer;
+import org.apache.distributedlog.io.TransmitListener;
+
+import java.io.IOException;
+
+/**
+ * Write representation of a {@link Entry}.
+ * It is a buffer of log record set, used for transmission.
+ */
+public interface EntryBuffer extends TransmitListener {
+
+ /**
+ * Return if this record set contains user records.
+ *
+ * @return true if this record set contains user records, otherwise
+ * return false.
+ */
+ boolean hasUserRecords();
+
+ /**
+ * Return number of records in current record set.
+ *
+ * @return number of records in current record set.
+ */
+ int getNumRecords();
+
+ /**
+ * Return number of bytes in current record set.
+ *
+ * @return number of bytes in current record set.
+ */
+ int getNumBytes();
+
+ /**
+ * Return max tx id in current record set.
+ *
+ * @return max tx id.
+ */
+ long getMaxTxId();
+
+ /**
+ * Get the buffer to transmit.
+ *
+ * @return the buffer to transmit.
+ * @throws InvalidEnvelopedEntryException if the record set buffer is invalid
+ * @throws IOException when encountered IOException during serialization
+ */
+ Buffer getBuffer() throws InvalidEnvelopedEntryException, IOException;
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EntryPosition.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EntryPosition.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EntryPosition.java
new file mode 100644
index 0000000..218662c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EntryPosition.java
@@ -0,0 +1,63 @@
+/**
+ * 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 org.apache.distributedlog;
+
+/**
+ * The position of an entry, identified by log segment sequence number and entry id.
+ */
+class EntryPosition {
+
+ private long lssn;
+ private long entryId;
+
+ EntryPosition(long lssn, long entryId) {
+ this.lssn = lssn;
+ this.entryId = entryId;
+ }
+
+ public synchronized long getLogSegmentSequenceNumber() {
+ return lssn;
+ }
+
+ public synchronized long getEntryId() {
+ return entryId;
+ }
+
+ public synchronized boolean advance(long lssn, long entryId) {
+ if (lssn == this.lssn) {
+ if (entryId <= this.entryId) {
+ return false;
+ }
+ this.entryId = entryId;
+ return true;
+ } else if (lssn > this.lssn) {
+ this.lssn = lssn;
+ this.entryId = entryId;
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("(").append(lssn).append(", ").append(entryId).append(")");
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
new file mode 100644
index 0000000..eb1e9af
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
@@ -0,0 +1,296 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+import org.apache.distributedlog.annotations.DistributedLogAnnotations.Compression;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.io.CompressionUtils;
+import org.apache.distributedlog.util.BitMaskUtils;
+
+/**
+ * An enveloped entry written to BookKeeper.
+ *
+ * Data type in brackets. Interpretation should be on the basis of data types and not individual
+ * bytes to honor Endianness.
+ *
+ * Entry Structure:
+ * ---------------
+ * Bytes 0 : Version (Byte)
+ * Bytes 1 - (DATA = 1+Header.length-1) : Header (Integer)
+ * Bytes DATA - DATA+3 : Payload Length (Integer)
+ * BYTES DATA+4 - DATA+4+payload.length-1 : Payload (Byte[])
+ *
+ * V1 Header Structure: // Offsets relative to the start of the header.
+ * -------------------
+ * Bytes 0 - 3 : Flags (Integer)
+ * Bytes 4 - 7 : Original payload size before compression (Integer)
+ *
+ * Flags: // 32 Bits
+ * -----
+ * 0 ... 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
+ * |_|
+ * |
+ * Compression Type
+ *
+ * Compression Type: // 2 Bits (Least significant)
+ * ----------------
+ * 00 : No Compression
+ * 01 : LZ4 Compression
+ * 10 : Unused
+ * 11 : Unused
+ */
+public class EnvelopedEntry {
+
+ public static final int VERSION_LENGTH = 1; // One byte long
+ public static final byte VERSION_ONE = 1;
+
+ public static final byte LOWEST_SUPPORTED_VERSION = VERSION_ONE;
+ public static final byte HIGHEST_SUPPORTED_VERSION = VERSION_ONE;
+ public static final byte CURRENT_VERSION = VERSION_ONE;
+
+ private final OpStatsLogger compressionStat;
+ private final OpStatsLogger decompressionStat;
+ private final Counter compressedEntryBytes;
+ private final Counter decompressedEntryBytes;
+ private final byte version;
+
+ private Header header = new Header();
+ private Payload payloadCompressed = new Payload();
+ private Payload payloadDecompressed = new Payload();
+
+ public EnvelopedEntry(byte version,
+ StatsLogger statsLogger) throws InvalidEnvelopedEntryException {
+ Preconditions.checkNotNull(statsLogger);
+ if (version < LOWEST_SUPPORTED_VERSION || version > HIGHEST_SUPPORTED_VERSION) {
+ throw new InvalidEnvelopedEntryException("Invalid enveloped entry version " + version + ", expected to be in [ "
+ + LOWEST_SUPPORTED_VERSION + " ~ " + HIGHEST_SUPPORTED_VERSION + " ]");
+ }
+ this.version = version;
+ this.compressionStat = statsLogger.getOpStatsLogger("compression_time");
+ this.decompressionStat = statsLogger.getOpStatsLogger("decompression_time");
+ this.compressedEntryBytes = statsLogger.getCounter("compressed_bytes");
+ this.decompressedEntryBytes = statsLogger.getCounter("decompressed_bytes");
+ }
+
+ /**
+ * @param statsLogger
+ * Used for getting stats for (de)compression time
+ * @param compressionType
+ * The compression type to use
+ * @param decompressed
+ * The decompressed payload
+ * NOTE: The size of the byte array passed as the decompressed payload can be larger
+ * than the actual contents to be compressed.
+ */
+ public EnvelopedEntry(byte version,
+ CompressionCodec.Type compressionType,
+ byte[] decompressed,
+ int length,
+ StatsLogger statsLogger)
+ throws InvalidEnvelopedEntryException {
+ this(version, statsLogger);
+ Preconditions.checkNotNull(compressionType);
+ Preconditions.checkNotNull(decompressed);
+ Preconditions.checkArgument(length >= 0, "Invalid bytes length " + length);
+
+ this.header = new Header(compressionType, length);
+ this.payloadDecompressed = new Payload(length, decompressed);
+ }
+
+ private boolean isReady() {
+ return (header.ready && payloadDecompressed.ready);
+ }
+
+ @Compression
+ public void writeFully(DataOutputStream out) throws IOException {
+ Preconditions.checkNotNull(out);
+ if (!isReady()) {
+ throw new IOException("Entry not writable");
+ }
+ // Version
+ out.writeByte(version);
+ // Header
+ header.write(out);
+ // Compress
+ CompressionCodec codec = CompressionUtils.getCompressionCodec(header.compressionType);
+ byte[] compressed = codec.compress(
+ payloadDecompressed.payload,
+ 0,
+ payloadDecompressed.length,
+ compressionStat);
+ this.payloadCompressed = new Payload(compressed.length, compressed);
+ this.compressedEntryBytes.add(payloadCompressed.length);
+ this.decompressedEntryBytes.add(payloadDecompressed.length);
+ payloadCompressed.write(out);
+ }
+
+ @Compression
+ public void readFully(DataInputStream in) throws IOException {
+ Preconditions.checkNotNull(in);
+ // Make sure we're reading the right versioned entry.
+ byte version = in.readByte();
+ if (version != this.version) {
+ throw new IOException(String.format("Version mismatch while reading. Received: %d," +
+ " Required: %d", version, this.version));
+ }
+ header.read(in);
+ payloadCompressed.read(in);
+ // Decompress
+ CompressionCodec codec = CompressionUtils.getCompressionCodec(header.compressionType);
+ byte[] decompressed = codec.decompress(
+ payloadCompressed.payload,
+ 0,
+ payloadCompressed.length,
+ header.decompressedSize,
+ decompressionStat);
+ this.payloadDecompressed = new Payload(decompressed.length, decompressed);
+ this.compressedEntryBytes.add(payloadCompressed.length);
+ this.decompressedEntryBytes.add(payloadDecompressed.length);
+ }
+
+ public byte[] getDecompressedPayload() throws IOException {
+ if (!isReady()) {
+ throw new IOException("Decompressed payload is not initialized");
+ }
+ return payloadDecompressed.payload;
+ }
+
+ public static class Header {
+ public static final int COMPRESSION_CODEC_MASK = 0x3;
+ public static final int COMPRESSION_CODEC_NONE = 0x0;
+ public static final int COMPRESSION_CODEC_LZ4 = 0x1;
+
+ private int flags = 0;
+ private int decompressedSize = 0;
+ private CompressionCodec.Type compressionType = CompressionCodec.Type.UNKNOWN;
+
+ // Whether this struct is ready for reading/writing.
+ private boolean ready = false;
+
+ // Used while reading.
+ public Header() {
+ }
+
+ public Header(CompressionCodec.Type compressionType,
+ int decompressedSize) {
+ this.compressionType = compressionType;
+ this.decompressedSize = decompressedSize;
+ this.flags = 0;
+ switch (compressionType) {
+ case NONE:
+ this.flags = (int) BitMaskUtils.set(flags, COMPRESSION_CODEC_MASK,
+ COMPRESSION_CODEC_NONE);
+ break;
+ case LZ4:
+ this.flags = (int) BitMaskUtils.set(flags, COMPRESSION_CODEC_MASK,
+ COMPRESSION_CODEC_LZ4);
+ break;
+ default:
+ throw new RuntimeException(String.format("Unknown Compression Type: %s",
+ compressionType));
+ }
+ // This can now be written.
+ this.ready = true;
+ }
+
+ private void write(DataOutputStream out) throws IOException {
+ out.writeInt(flags);
+ out.writeInt(decompressedSize);
+ }
+
+ private void read(DataInputStream in) throws IOException {
+ this.flags = in.readInt();
+ int compressionType = (int) BitMaskUtils.get(flags, COMPRESSION_CODEC_MASK);
+ if (compressionType == COMPRESSION_CODEC_NONE) {
+ this.compressionType = CompressionCodec.Type.NONE;
+ } else if (compressionType == COMPRESSION_CODEC_LZ4) {
+ this.compressionType = CompressionCodec.Type.LZ4;
+ } else {
+ throw new IOException(String.format("Unsupported Compression Type: %s",
+ compressionType));
+ }
+ this.decompressedSize = in.readInt();
+ // Values can now be read.
+ this.ready = true;
+ }
+ }
+
+ public static class Payload {
+ private int length = 0;
+ private byte[] payload = null;
+
+ // Whether this struct is ready for reading/writing.
+ private boolean ready = false;
+
+ // Used for reading
+ Payload() {
+ }
+
+ Payload(int length, byte[] payload) {
+ this.length = length;
+ this.payload = payload;
+ this.ready = true;
+ }
+
+ private void write(DataOutputStream out) throws IOException {
+ out.writeInt(length);
+ out.write(payload, 0, length);
+ }
+
+ private void read(DataInputStream in) throws IOException {
+ this.length = in.readInt();
+ this.payload = new byte[length];
+ in.readFully(payload);
+ this.ready = true;
+ }
+ }
+
+ /**
+ * Return an InputStream that reads from the provided InputStream, decompresses the data
+ * and returns a new InputStream wrapping the underlying payload.
+ *
+ * Note that src is modified by this call.
+ *
+ * @return
+ * New Input stream with the underlying payload.
+ * @throws Exception
+ */
+ public static InputStream fromInputStream(InputStream src,
+ StatsLogger statsLogger) throws IOException {
+ src.mark(VERSION_LENGTH);
+ byte version = new DataInputStream(src).readByte();
+ src.reset();
+ EnvelopedEntry entry = new EnvelopedEntry(version, statsLogger);
+ entry.readFully(new DataInputStream(src));
+ return new ByteArrayInputStream(entry.getDecompressedPayload());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java
new file mode 100644
index 0000000..1761de5
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java
@@ -0,0 +1,102 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.bookkeeper.stats.StatsLogger;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Record reader to read records from an enveloped entry buffer.
+ */
+class EnvelopedEntryReader implements Entry.Reader, RecordStream {
+
+ private final long logSegmentSeqNo;
+ private final long entryId;
+ private final LogRecord.Reader reader;
+
+ // slot id
+ private long slotId = 0;
+
+ EnvelopedEntryReader(long logSegmentSeqNo,
+ long entryId,
+ long startSequenceId,
+ InputStream in,
+ boolean envelopedEntry,
+ boolean deserializeRecordSet,
+ StatsLogger statsLogger)
+ throws IOException {
+ this.logSegmentSeqNo = logSegmentSeqNo;
+ this.entryId = entryId;
+ InputStream src = in;
+ if (envelopedEntry) {
+ src = EnvelopedEntry.fromInputStream(in, statsLogger);
+ }
+ this.reader = new LogRecord.Reader(
+ this,
+ new DataInputStream(src),
+ startSequenceId,
+ deserializeRecordSet);
+ }
+
+ @Override
+ public long getLSSN() {
+ return logSegmentSeqNo;
+ }
+
+ @Override
+ public long getEntryId() {
+ return entryId;
+ }
+
+ @Override
+ public LogRecordWithDLSN nextRecord() throws IOException {
+ return reader.readOp();
+ }
+
+ @Override
+ public boolean skipTo(long txId) throws IOException {
+ return reader.skipTo(txId, true);
+ }
+
+ @Override
+ public boolean skipTo(DLSN dlsn) throws IOException {
+ return reader.skipTo(dlsn);
+ }
+
+ //
+ // Record Stream
+ //
+
+ @Override
+ public void advance(int numRecords) {
+ slotId += numRecords;
+ }
+
+ @Override
+ public DLSN getCurrentPosition() {
+ return new DLSN(logSegmentSeqNo, entryId, slotId);
+ }
+
+ @Override
+ public String getName() {
+ return "EnvelopedReader";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
new file mode 100644
index 0000000..54858d7
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
@@ -0,0 +1,192 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.Entry.Writer;
+import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.WriteCancelledException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.io.Buffer;
+import org.apache.distributedlog.io.CompressionCodec;
+import com.twitter.util.Promise;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+
+/**
+ * {@link org.apache.distributedlog.io.Buffer} based log record set writer.
+ */
+class EnvelopedEntryWriter implements Writer {
+
+ static final Logger logger = LoggerFactory.getLogger(EnvelopedEntryWriter.class);
+
+ private static class WriteRequest {
+
+ private final int numRecords;
+ private final Promise<DLSN> promise;
+
+ WriteRequest(int numRecords, Promise<DLSN> promise) {
+ this.numRecords = numRecords;
+ this.promise = promise;
+ }
+
+ }
+
+ private final String logName;
+ private final Buffer buffer;
+ private final LogRecord.Writer writer;
+ private final List<WriteRequest> writeRequests;
+ private final boolean envelopeBeforeTransmit;
+ private final CompressionCodec.Type codec;
+ private final StatsLogger statsLogger;
+ private int count = 0;
+ private boolean hasUserData = false;
+ private long maxTxId = Long.MIN_VALUE;
+
+ EnvelopedEntryWriter(String logName,
+ int initialBufferSize,
+ boolean envelopeBeforeTransmit,
+ CompressionCodec.Type codec,
+ StatsLogger statsLogger) {
+ this.logName = logName;
+ this.buffer = new Buffer(initialBufferSize * 6 / 5);
+ this.writer = new LogRecord.Writer(new DataOutputStream(buffer));
+ this.writeRequests = new LinkedList<WriteRequest>();
+ this.envelopeBeforeTransmit = envelopeBeforeTransmit;
+ this.codec = codec;
+ this.statsLogger = statsLogger;
+ }
+
+ @Override
+ public synchronized void reset() {
+ cancelPromises(new WriteCancelledException(logName, "Record Set is reset"));
+ count = 0;
+ this.buffer.reset();
+ }
+
+ @Override
+ public synchronized void writeRecord(LogRecord record,
+ Promise<DLSN> transmitPromise)
+ throws LogRecordTooLongException, WriteException {
+ int logRecordSize = record.getPersistentSize();
+ if (logRecordSize > MAX_LOGRECORD_SIZE) {
+ throw new LogRecordTooLongException(
+ "Log Record of size " + logRecordSize + " written when only "
+ + MAX_LOGRECORD_SIZE + " is allowed");
+ }
+
+ try {
+ this.writer.writeOp(record);
+ int numRecords = 1;
+ if (!record.isControl()) {
+ hasUserData = true;
+ }
+ if (record.isRecordSet()) {
+ numRecords = LogRecordSet.numRecords(record);
+ }
+ count += numRecords;
+ writeRequests.add(new WriteRequest(numRecords, transmitPromise));
+ maxTxId = Math.max(maxTxId, record.getTransactionId());
+ } catch (IOException e) {
+ logger.error("Failed to append record to record set of {} : ",
+ logName, e);
+ throw new WriteException(logName, "Failed to append record to record set of "
+ + logName);
+ }
+ }
+
+ private synchronized void satisfyPromises(long lssn, long entryId) {
+ long nextSlotId = 0;
+ for (WriteRequest request : writeRequests) {
+ request.promise.setValue(new DLSN(lssn, entryId, nextSlotId));
+ nextSlotId += request.numRecords;
+ }
+ writeRequests.clear();
+ }
+
+ private synchronized void cancelPromises(Throwable reason) {
+ for (WriteRequest request : writeRequests) {
+ request.promise.setException(reason);
+ }
+ writeRequests.clear();
+ }
+
+ @Override
+ public synchronized long getMaxTxId() {
+ return maxTxId;
+ }
+
+ @Override
+ public synchronized boolean hasUserRecords() {
+ return hasUserData;
+ }
+
+ @Override
+ public int getNumBytes() {
+ return buffer.size();
+ }
+
+ @Override
+ public synchronized int getNumRecords() {
+ return count;
+ }
+
+ @Override
+ public synchronized Buffer getBuffer() throws InvalidEnvelopedEntryException, IOException {
+ if (!envelopeBeforeTransmit) {
+ return buffer;
+ }
+ // We can't escape this allocation because things need to be read from one byte array
+ // and then written to another. This is the destination.
+ Buffer toSend = new Buffer(buffer.size());
+ byte[] decompressed = buffer.getData();
+ int length = buffer.size();
+ EnvelopedEntry entry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
+ codec,
+ decompressed,
+ length,
+ statsLogger);
+ // This will cause an allocation of a byte[] for compression. This can be avoided
+ // but we can do that later only if needed.
+ entry.writeFully(new DataOutputStream(toSend));
+ return toSend;
+ }
+
+ @Override
+ public synchronized DLSN finalizeTransmit(long lssn, long entryId) {
+ return new DLSN(lssn, entryId, count - 1);
+ }
+
+ @Override
+ public void completeTransmit(long lssn, long entryId) {
+ satisfyPromises(lssn, entryId);
+ }
+
+ @Override
+ public void abortTransmit(Throwable reason) {
+ cancelPromises(reason);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LedgerReadPosition.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LedgerReadPosition.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LedgerReadPosition.java
new file mode 100644
index 0000000..f94495f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LedgerReadPosition.java
@@ -0,0 +1,171 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LedgerReadPosition {
+ static final Logger LOG = LoggerFactory.getLogger(LedgerReadPosition.class);
+
+ private static enum PartialOrderingComparisonResult {
+ NotComparable,
+ GreaterThan,
+ LessThan,
+ EqualTo
+ }
+
+ long ledgerId = DistributedLogConstants.UNRESOLVED_LEDGER_ID;
+ long logSegmentSequenceNo;
+ long entryId;
+
+ public LedgerReadPosition(long ledgerId, long logSegmentSequenceNo, long entryId) {
+ this.ledgerId = ledgerId;
+ this.logSegmentSequenceNo = logSegmentSequenceNo;
+ this.entryId = entryId;
+ }
+
+ public LedgerReadPosition(LedgerReadPosition that) {
+ this.ledgerId = that.ledgerId;
+ this.logSegmentSequenceNo = that.logSegmentSequenceNo;
+ this.entryId = that.entryId;
+ }
+
+
+ public LedgerReadPosition(final DLSN dlsn) {
+ this(dlsn.getLogSegmentSequenceNo(), dlsn.getEntryId());
+ }
+
+ public LedgerReadPosition(long logSegmentSequenceNo, long entryId) {
+ this.logSegmentSequenceNo = logSegmentSequenceNo;
+ this.entryId = entryId;
+ }
+
+ public long getLedgerId() {
+ if (DistributedLogConstants.UNRESOLVED_LEDGER_ID == ledgerId) {
+ LOG.trace("Ledger Id is not initialized");
+ throw new IllegalStateException("Ledger Id is not initialized");
+ }
+ return ledgerId;
+ }
+
+ public long getLogSegmentSequenceNumber() {
+ return logSegmentSequenceNo;
+ }
+
+ public long getEntryId() {
+ return entryId;
+ }
+
+ public void advance() {
+ entryId++;
+ }
+
+ public void positionOnNewLogSegment(long ledgerId, long logSegmentSequenceNo) {
+ this.ledgerId = ledgerId;
+ this.logSegmentSequenceNo = logSegmentSequenceNo;
+ this.entryId = 0L;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("(lid=%d, lseqNo=%d, eid=%d)", ledgerId, logSegmentSequenceNo, entryId);
+ }
+
+ public boolean definitelyLessThanOrEqualTo(LedgerReadPosition threshold) {
+ PartialOrderingComparisonResult result = comparePartiallyOrdered(threshold);
+ return ((result == PartialOrderingComparisonResult.LessThan) ||
+ (result == PartialOrderingComparisonResult.EqualTo));
+ }
+
+ public boolean definitelyLessThan(LedgerReadPosition threshold) {
+ PartialOrderingComparisonResult result = comparePartiallyOrdered(threshold);
+ return result == PartialOrderingComparisonResult.LessThan;
+ }
+
+ private PartialOrderingComparisonResult comparePartiallyOrdered(LedgerReadPosition threshold) {
+ // If no threshold is passed we cannot make a definitive comparison
+ if (null == threshold) {
+ return PartialOrderingComparisonResult.NotComparable;
+ }
+
+ if (this.logSegmentSequenceNo != threshold.logSegmentSequenceNo) {
+ if (this.logSegmentSequenceNo < threshold.logSegmentSequenceNo) {
+ return PartialOrderingComparisonResult.LessThan;
+ } else {
+ return PartialOrderingComparisonResult.GreaterThan;
+ }
+ } else if (this.ledgerId != threshold.ledgerId) {
+ // When logSegmentSequenceNo is equal we cannot definitely say that this
+ // position is less than the threshold unless ledgerIds are equal
+ // since LogSegmentSequenceNumber maybe inferred from transactionIds in older
+ // versions of the metadata.
+ return PartialOrderingComparisonResult.NotComparable;
+ } else if (this.getEntryId() < threshold.getEntryId()) {
+ return PartialOrderingComparisonResult.LessThan;
+ } else if (this.getEntryId() > threshold.getEntryId()) {
+ return PartialOrderingComparisonResult.GreaterThan;
+ } else {
+ return PartialOrderingComparisonResult.EqualTo;
+ }
+ }
+
+ /**
+ * Comparator for the key portion
+ */
+ public static final ReadAheadCacheKeyComparator COMPARATOR = new ReadAheadCacheKeyComparator();
+
+ // Only compares the key portion
+ @Override
+ public boolean equals(Object other) {
+ if (!(other instanceof LedgerReadPosition)) {
+ return false;
+ }
+ LedgerReadPosition key = (LedgerReadPosition) other;
+ return ledgerId == key.ledgerId &&
+ entryId == key.entryId;
+ }
+
+ @Override
+ public int hashCode() {
+ return (int) (ledgerId * 13 ^ entryId * 17);
+ }
+
+ /**
+ * Compare EntryKey.
+ */
+ protected static class ReadAheadCacheKeyComparator implements Comparator<LedgerReadPosition>, Serializable {
+
+ private static final long serialVersionUID = 0L;
+
+ @Override
+ public int compare(LedgerReadPosition left, LedgerReadPosition right) {
+ long ret = left.ledgerId - right.ledgerId;
+ if (ret == 0) {
+ ret = left.entryId - right.entryId;
+ }
+ return (ret < 0) ? -1 : ((ret > 0) ? 1 : 0);
+ }
+ }
+
+}
+
+
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
new file mode 100644
index 0000000..5623525
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
@@ -0,0 +1,364 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Optional;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.bookkeeper.util.LocalBookKeeper;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.BindException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Utility class for setting up bookkeeper ensembles
+ * and bringing individual bookies up and down
+ */
+public class LocalDLMEmulator {
+ private static final Logger LOG = LoggerFactory.getLogger(LocalDLMEmulator.class);
+
+ public static final String DLOG_NAMESPACE = "/messaging/distributedlog";
+
+ private static final int DEFAULT_BOOKIE_INITIAL_PORT = 0; // Use ephemeral ports
+ private static final int DEFAULT_ZK_TIMEOUT_SEC = 10;
+ private static final int DEFAULT_ZK_PORT = 2181;
+ private static final String DEFAULT_ZK_HOST = "127.0.0.1";
+ private static final String DEFAULT_ZK_ENSEMBLE = DEFAULT_ZK_HOST + ":" + DEFAULT_ZK_PORT;
+ private static final int DEFAULT_NUM_BOOKIES = 3;
+ private static final ServerConfiguration DEFAULT_SERVER_CONFIGURATION = new ServerConfiguration();
+
+ private final String zkEnsemble;
+ private final URI uri;
+ private final List<File> tmpDirs = new ArrayList<File>();
+ private final int zkTimeoutSec;
+ private final Thread bkStartupThread;
+ private final String zkHost;
+ private final int zkPort;
+ private final int numBookies;
+
+ public static class Builder {
+ private int zkTimeoutSec = DEFAULT_ZK_TIMEOUT_SEC;
+ private int numBookies = DEFAULT_NUM_BOOKIES;
+ private String zkHost = DEFAULT_ZK_HOST;
+ private int zkPort = DEFAULT_ZK_PORT;
+ private int initialBookiePort = DEFAULT_BOOKIE_INITIAL_PORT;
+ private boolean shouldStartZK = true;
+ private Optional<ServerConfiguration> serverConf = Optional.absent();
+
+ public Builder numBookies(int numBookies) {
+ this.numBookies = numBookies;
+ return this;
+ }
+ public Builder zkHost(String zkHost) {
+ this.zkHost = zkHost;
+ return this;
+ }
+ public Builder zkPort(int zkPort) {
+ this.zkPort = zkPort;
+ return this;
+ }
+ public Builder zkTimeoutSec(int zkTimeoutSec) {
+ this.zkTimeoutSec = zkTimeoutSec;
+ return this;
+ }
+ public Builder initialBookiePort(int initialBookiePort) {
+ this.initialBookiePort = initialBookiePort;
+ return this;
+ }
+ public Builder shouldStartZK(boolean shouldStartZK) {
+ this.shouldStartZK = shouldStartZK;
+ return this;
+ }
+ public Builder serverConf(ServerConfiguration serverConf) {
+ this.serverConf = Optional.of(serverConf);
+ return this;
+ }
+
+ public LocalDLMEmulator build() throws Exception {
+ ServerConfiguration conf = null;
+ if (serverConf.isPresent()) {
+ conf = serverConf.get();
+ } else {
+ conf = (ServerConfiguration) DEFAULT_SERVER_CONFIGURATION.clone();
+ conf.setZkTimeout(zkTimeoutSec * 1000);
+ }
+ ServerConfiguration newConf = new ServerConfiguration();
+ newConf.loadConf(conf);
+ newConf.setAllowLoopback(true);
+
+ return new LocalDLMEmulator(numBookies, shouldStartZK, zkHost, zkPort,
+ initialBookiePort, zkTimeoutSec, newConf);
+ }
+ }
+
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ private LocalDLMEmulator(final int numBookies, final boolean shouldStartZK, final String zkHost, final int zkPort, final int initialBookiePort, final int zkTimeoutSec, final ServerConfiguration serverConf) throws Exception {
+ this.numBookies = numBookies;
+ this.zkHost = zkHost;
+ this.zkPort = zkPort;
+ this.zkEnsemble = zkHost + ":" + zkPort;
+ this.uri = URI.create("distributedlog://" + zkEnsemble + DLOG_NAMESPACE);
+ this.zkTimeoutSec = zkTimeoutSec;
+ this.bkStartupThread = new Thread() {
+ public void run() {
+ try {
+ LOG.info("Starting {} bookies : allowLoopback = {}", numBookies, serverConf.getAllowLoopback());
+ LocalBookKeeper.startLocalBookies(zkHost, zkPort, numBookies, shouldStartZK, initialBookiePort, serverConf);
+ LOG.info("{} bookies are started.");
+ } catch (InterruptedException e) {
+ // go away quietly
+ } catch (Exception e) {
+ LOG.error("Error starting local bk", e);
+ }
+ }
+ };
+ }
+
+ public void start() throws Exception {
+ bkStartupThread.start();
+ if (!LocalBookKeeper.waitForServerUp(zkEnsemble, zkTimeoutSec*1000)) {
+ throw new Exception("Error starting zookeeper/bookkeeper");
+ }
+ int bookiesUp = checkBookiesUp(numBookies, zkTimeoutSec);
+ assert (numBookies == bookiesUp);
+ // Provision "/messaging/distributedlog" namespace
+ DLMetadata.create(new BKDLConfig(zkEnsemble, "/ledgers")).create(uri);
+ }
+
+ public void teardown() throws Exception {
+ if (bkStartupThread != null) {
+ bkStartupThread.interrupt();
+ bkStartupThread.join();
+ }
+ for (File dir : tmpDirs) {
+ FileUtils.deleteDirectory(dir);
+ }
+ }
+
+ public String getZkServers() {
+ return zkEnsemble;
+ }
+
+ public URI getUri() {
+ return uri;
+ }
+
+ public BookieServer newBookie() throws Exception {
+ ServerConfiguration bookieConf = new ServerConfiguration();
+ bookieConf.setZkTimeout(zkTimeoutSec * 1000);
+ bookieConf.setBookiePort(0);
+ bookieConf.setAllowLoopback(true);
+ File tmpdir = File.createTempFile("bookie" + UUID.randomUUID() + "_",
+ "test");
+ if (!tmpdir.delete()) {
+ LOG.debug("Fail to delete tmpdir " + tmpdir);
+ }
+ if (!tmpdir.mkdir()) {
+ throw new IOException("Fail to create tmpdir " + tmpdir);
+ }
+ tmpDirs.add(tmpdir);
+
+ bookieConf.setZkServers(zkEnsemble);
+ bookieConf.setJournalDirName(tmpdir.getPath());
+ bookieConf.setLedgerDirNames(new String[]{tmpdir.getPath()});
+
+ BookieServer b = new BookieServer(bookieConf);
+ b.start();
+ for (int i = 0; i < 10 && !b.isRunning(); i++) {
+ Thread.sleep(10000);
+ }
+ if (!b.isRunning()) {
+ throw new IOException("Bookie would not start");
+ }
+ return b;
+ }
+
+ /**
+ * Check that a number of bookies are available
+ *
+ * @param count number of bookies required
+ * @param timeout number of seconds to wait for bookies to start
+ * @throws java.io.IOException if bookies are not started by the time the timeout hits
+ */
+ public int checkBookiesUp(int count, int timeout) throws Exception {
+ ZooKeeper zkc = connectZooKeeper(zkHost, zkPort, zkTimeoutSec);
+ try {
+ int mostRecentSize = 0;
+ for (int i = 0; i < timeout; i++) {
+ try {
+ List<String> children = zkc.getChildren("/ledgers/available",
+ false);
+ children.remove("readonly");
+ mostRecentSize = children.size();
+ if ((mostRecentSize > count) || LOG.isDebugEnabled()) {
+ LOG.info("Found " + mostRecentSize + " bookies up, "
+ + "waiting for " + count);
+ if ((mostRecentSize > count) || LOG.isTraceEnabled()) {
+ for (String child : children) {
+ LOG.info(" server: " + child);
+ }
+ }
+ }
+ if (mostRecentSize == count) {
+ break;
+ }
+ } catch (KeeperException e) {
+ // ignore
+ }
+ Thread.sleep(1000);
+ }
+ return mostRecentSize;
+ } finally {
+ zkc.close();
+ }
+ }
+
+ public static String getBkLedgerPath() {
+ return "/ledgers";
+ }
+
+ public static ZooKeeper connectZooKeeper(String zkHost, int zkPort)
+ throws IOException, KeeperException, InterruptedException {
+ return connectZooKeeper(zkHost, zkPort, DEFAULT_ZK_TIMEOUT_SEC);
+ }
+
+ public static ZooKeeper connectZooKeeper(String zkHost, int zkPort, int zkTimeoutSec)
+ throws IOException, KeeperException, InterruptedException {
+ final CountDownLatch latch = new CountDownLatch(1);
+ final String zkHostPort = zkHost + ":" + zkPort;
+
+ ZooKeeper zkc = new ZooKeeper(zkHostPort, zkTimeoutSec * 1000, new Watcher() {
+ public void process(WatchedEvent event) {
+ if (event.getState() == Event.KeeperState.SyncConnected) {
+ latch.countDown();
+ }
+ }
+ });
+ if (!latch.await(zkTimeoutSec, TimeUnit.SECONDS)) {
+ throw new IOException("Zookeeper took too long to connect");
+ }
+ return zkc;
+ }
+
+ public static URI createDLMURI(String path) throws Exception {
+ return createDLMURI(DEFAULT_ZK_ENSEMBLE, path);
+ }
+
+ public static URI createDLMURI(String zkServers, String path) throws Exception {
+ return URI.create("distributedlog://" + zkServers + DLOG_NAMESPACE + path);
+ }
+
+ /**
+ * Try to start zookkeeper locally on any port.
+ */
+ public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(File zkDir) throws Exception {
+ return runZookeeperOnAnyPort((int) (Math.random()*10000+7000), zkDir);
+ }
+
+ /**
+ * Try to start zookkeeper locally on any port beginning with some base port.
+ * Dump some socket info when bind fails.
+ */
+ public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(int basePort, File zkDir) throws Exception {
+
+ final int MAX_RETRIES = 20;
+ final int MIN_PORT = 1025;
+ final int MAX_PORT = 65535;
+ ZooKeeperServerShim zks = null;
+ int zkPort = basePort;
+ boolean success = false;
+ int retries = 0;
+
+ while (!success) {
+ try {
+ LOG.info("zk trying to bind to port " + zkPort);
+ zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkDir);
+ success = true;
+ } catch (BindException be) {
+ retries++;
+ if (retries > MAX_RETRIES) {
+ throw be;
+ }
+ zkPort++;
+ if (zkPort > MAX_PORT) {
+ zkPort = MIN_PORT;
+ }
+ }
+ }
+
+ return Pair.of(zks, zkPort);
+ }
+
+ public static void main(String[] args) throws Exception {
+ try {
+ if (args.length < 1) {
+ System.out.println("Usage: LocalDLEmulator <zk_port>");
+ System.exit(-1);
+ }
+
+ final int zkPort = Integer.parseInt(args[0]);
+ final File zkDir = IOUtils.createTempDir("distrlog", "zookeeper");
+ final LocalDLMEmulator localDlm = LocalDLMEmulator.newBuilder()
+ .zkPort(zkPort)
+ .build();
+
+ Runtime.getRuntime().addShutdownHook(new Thread() {
+ @Override
+ public void run() {
+ try {
+ localDlm.teardown();
+ FileUtils.deleteDirectory(zkDir);
+ System.out.println("ByeBye!");
+ } catch (Exception e) {
+ // do nothing
+ }
+ }
+ });
+ localDlm.start();
+
+ System.out.println(String.format(
+ "DistributedLog Sandbox is running now. You could access distributedlog://%s:%s",
+ DEFAULT_ZK_HOST,
+ zkPort));
+ } catch (Exception ex) {
+ System.out.println("Exception occurred running emulator " + ex);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
new file mode 100644
index 0000000..75a32ef
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
@@ -0,0 +1,195 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.io.AsyncCloseable;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * <i>LogReader</i> is a `synchronous` reader reading records from a DL log.
+ *
+ * <h3>Lifecycle of a Reader</h3>
+ *
+ * A reader is a <i>sequential</i> reader that read records from a DL log starting
+ * from a given position. The position could be a <i>DLSN</i> (via {@link DistributedLogManager#getInputStream(DLSN)}
+ * or a <i>Transaction ID</i> (via {@link DistributedLogManager#getInputStream(long)}.
+ * <p>
+ * After the reader is open, it could call {@link #readNext(boolean)} or {@link #readBulk(boolean, int)}
+ * to read records out the log from provided position.
+ * <p>
+ * Closing the reader (via {@link #close()} will release all the resources occupied
+ * by this reader instance.
+ * <p>
+ * Exceptions could be thrown during reading records. Once the exception is thrown,
+ * the reader is set to an error state and it isn't usable anymore. It is the application's
+ * responsibility to handle the exceptions and re-create readers if necessary.
+ * <p>
+ * Example:
+ * <pre>
+ * DistributedLogManager dlm = ...;
+ * long nextTxId = ...;
+ * LogReader reader = dlm.getInputStream(nextTxId);
+ *
+ * while (true) { // keep reading & processing records
+ * LogRecord record;
+ * try {
+ * record = reader.readNext(false);
+ * nextTxId = record.getTransactionId();
+ * // process the record
+ * ...
+ * } catch (IOException ioe) {
+ * // handle the exception
+ * ...
+ * reader = dlm.getInputStream(nextTxId + 1);
+ * }
+ * }
+ *
+ * </pre>
+ *
+ * <h3>Read Records</h3>
+ *
+ * Reading records from an <i>endless</i> log in `synchronous` way isn't as
+ * trivial as in `asynchronous` way (via {@link AsyncLogReader}. Because it
+ * lacks of callback mechanism. LogReader introduces a flag `nonBlocking` on
+ * controlling the <i>waiting</i> behavior on `synchronous` reads.
+ *
+ * <h4>Blocking vs NonBlocking</h4>
+ *
+ * <i>Blocking</i> (nonBlocking = false) means the reads will wait for records
+ * before returning read calls. While <i>NonBlocking</i> (nonBlocking = true)
+ * means the reads will only check readahead cache and return whatever records
+ * available in the readahead cache.
+ * <p>
+ * The <i>waiting</i> period varies in <i>blocking</i> mode. If the reader is
+ * catching up with writer (there are records in the log), the read call will
+ * wait until records are read and returned. If the reader is caught up with
+ * writer (there are no more records in the log at read time), the read call
+ * will wait for a small period of time (defined in
+ * {@link DistributedLogConfiguration#getReadAheadWaitTime()} and return whatever
+ * records available in the readahead cache. In other words, if a reader sees
+ * no record on blocking reads, it means the reader is `caught-up` with the
+ * writer.
+ * <p>
+ * <i>Blocking</i> and <i>NonBlocking</i> modes are useful for building replicated
+ * state machines. Applications could use <i>blocking</i> reads till caught up
+ * with latest data. Once they are caught up with latest data, they could start
+ * serving their service and turn to <i>non-blocking</i> read mode and tail read
+ * data from the logs.
+ * <p>
+ * See examples below.
+ *
+ * <h4>Read Single Record</h4>
+ *
+ * {@link #readNext(boolean)} is reading individual records from a DL log.
+ *
+ * <pre>
+ * LogReader reader = ...
+ *
+ * // keep reading records in blocking way until no records available in the log
+ * LogRecord record = reader.readNext(false);
+ * while (null != record) {
+ * // process the record
+ * ...
+ * // read next record
+ * records = reader.readNext(false);
+ * }
+ *
+ * ...
+ *
+ * // reader is caught up with writer, doing non-blocking reads to tail the log
+ * while (true) {
+ * record = reader.readNext(true)
+ * // process the new records
+ * ...
+ * }
+ * </pre>
+ *
+ * <h4>Read Batch of Records</h4>
+ *
+ * {@link #readBulk(boolean, int)} is a convenient way to read a batch of records
+ * from a DL log.
+ *
+ * <pre>
+ * LogReader reader = ...
+ * int N = 10;
+ *
+ * // keep reading N records in blocking way until no records available in the log
+ * List<LogRecord> records = reader.readBulk(false, N);
+ * while (!records.isEmpty()) {
+ * // process the list of records
+ * ...
+ * if (records.size() < N) { // no more records available in the log
+ * break;
+ * }
+ * // read next N records
+ * records = reader.readBulk(false, N);
+ * }
+ *
+ * ...
+ *
+ * // reader is caught up with writer, doing non-blocking reads to tail the log
+ * while (true) {
+ * records = reader.readBulk(true, N)
+ * // process the new records
+ * ...
+ * }
+ *
+ * </pre>
+ *
+ * @see AsyncLogReader
+ *
+ * NOTE:
+ * 1. Extending {@link AsyncCloseable}: BKSyncLogReader is implemented based on BKAsyncLogReader, exposing
+ * the {@link AsyncCloseable} interface so the reader could be closed asynchronously
+ */
+public interface LogReader extends Closeable, AsyncCloseable {
+
+ /**
+ * Read the next log record from the stream.
+ * <p>
+ * If <i>nonBlocking</i> is set to true, the call returns immediately by just polling
+ * records from read ahead cache. It would return <i>null</i> if there isn't any records
+ * available in the read ahead cache.
+ * <p>
+ * If <i>nonBlocking</i> is set to false, it would does blocking call. The call will
+ * block until return a record if there are records in the stream (aka catching up).
+ * Otherwise it would wait up to {@link DistributedLogConfiguration#getReadAheadWaitTime()}
+ * milliseconds and return null if there isn't any more records in the stream.
+ *
+ * @param nonBlocking should the read make blocking calls to the backend or rely on the
+ * readAhead cache
+ * @return an operation from the stream or null if at end of stream
+ * @throws IOException if there is an error reading from the stream
+ */
+ public LogRecordWithDLSN readNext(boolean nonBlocking) throws IOException;
+
+ /**
+ * Read the next <i>numLogRecords</i> log records from the stream
+ *
+ * @param nonBlocking should the read make blocking calls to the backend or rely on the
+ * readAhead cache
+ * @param numLogRecords maximum number of log records returned by this call.
+ * @return an operation from the stream or empty list if at end of stream
+ * @throws IOException if there is an error reading from the stream
+ * @see #readNext(boolean)
+ */
+ public List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords) throws IOException;
+}
[22/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 91e6dec..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
+++ /dev/null
@@ -1,286 +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.logsegment;
-
-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;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * BookKeeper Based Entry Store
- */
-public class BKLogSegmentEntryStore implements
- LogSegmentEntryStore,
- AsyncCallback.OpenCallback,
- AsyncCallback.DeleteCallback {
-
- private static final Logger logger = LoggerFactory.getLogger(BKLogSegmentEntryReader.class);
-
- private static class OpenReaderRequest {
-
- private final LogSegmentMetadata segment;
- private final long startEntryId;
- private final Promise<LogSegmentEntryReader> openPromise;
-
- OpenReaderRequest(LogSegmentMetadata segment,
- long startEntryId) {
- this.segment = segment;
- this.startEntryId = startEntryId;
- this.openPromise = new Promise<LogSegmentEntryReader>();
- }
-
- }
-
- private static class DeleteLogSegmentRequest {
-
- private final LogSegmentMetadata segment;
- private final Promise<LogSegmentMetadata> deletePromise;
-
- DeleteLogSegmentRequest(LogSegmentMetadata segment) {
- this.segment = segment;
- this.deletePromise = new Promise<LogSegmentMetadata>();
- }
-
- }
-
- 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;
- }
-
- @Override
- public Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment) {
- DeleteLogSegmentRequest request = new DeleteLogSegmentRequest(segment);
- BookKeeper bk;
- try {
- bk = this.bkc.get();
- } catch (IOException e) {
- return Future.exception(e);
- }
- bk.asyncDeleteLedger(segment.getLogSegmentId(), this, request);
- return request.deletePromise;
- }
-
- @Override
- public void deleteComplete(int rc, Object ctx) {
- DeleteLogSegmentRequest deleteRequest = (DeleteLogSegmentRequest) ctx;
- if (BKException.Code.NoSuchLedgerExistsException == rc) {
- logger.warn("No ledger {} found to delete for {}.",
- deleteRequest.segment.getLogSegmentId(), deleteRequest.segment);
- } else if (BKException.Code.OK != rc) {
- logger.error("Couldn't delete ledger {} from bookkeeper for {} : {}",
- new Object[]{ deleteRequest.segment.getLogSegmentId(), deleteRequest.segment,
- BKException.getMessage(rc) });
- FutureUtils.setException(deleteRequest.deletePromise,
- new BKTransmitException("Couldn't delete log segment " + deleteRequest.segment, rc));
- return;
- }
- FutureUtils.setValue(deleteRequest.deletePromise, deleteRequest.segment);
- }
-
- //
- // 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;
- try {
- bk = this.bkc.get();
- } catch (IOException e) {
- return Future.exception(e);
- }
- OpenReaderRequest request = new OpenReaderRequest(segment, startEntryId);
- if (segment.isInProgress()) {
- bk.asyncOpenLedgerNoRecovery(
- segment.getLogSegmentId(),
- BookKeeper.DigestType.CRC32,
- passwd,
- this,
- request);
- } else {
- bk.asyncOpenLedger(
- segment.getLogSegmentId(),
- BookKeeper.DigestType.CRC32,
- passwd,
- this,
- request);
- }
- return request.openPromise;
- }
-
- @Override
- public void openComplete(int rc, LedgerHandle lh, Object ctx) {
- OpenReaderRequest request = (OpenReaderRequest) ctx;
- if (BKException.Code.OK != rc) {
- FutureUtils.setException(
- request.openPromise,
- new BKTransmitException("Failed to open ledger handle for log segment " + request.segment, rc));
- return;
- }
- // successfully open a ledger
- try {
- LogSegmentEntryReader reader = new BKLogSegmentEntryReader(
- request.segment,
- lh,
- request.startEntryId,
- bkc.get(),
- scheduler,
- conf,
- statsLogger,
- failureInjector);
- FutureUtils.setValue(request.openPromise, reader);
- } catch (IOException e) {
- FutureUtils.setException(request.openPromise, e);
- }
-
- }
-
- @Override
- public Future<LogSegmentRandomAccessEntryReader> openRandomAccessReader(final LogSegmentMetadata segment,
- final boolean fence) {
- final BookKeeper bk;
- try {
- bk = this.bkc.get();
- } catch (IOException e) {
- return Future.exception(e);
- }
- final Promise<LogSegmentRandomAccessEntryReader> openPromise = new Promise<LogSegmentRandomAccessEntryReader>();
- AsyncCallback.OpenCallback openCallback = new AsyncCallback.OpenCallback() {
- @Override
- public void openComplete(int rc, LedgerHandle lh, Object ctx) {
- if (BKException.Code.OK != rc) {
- FutureUtils.setException(
- openPromise,
- new BKTransmitException("Failed to open ledger handle for log segment " + segment, rc));
- return;
- }
- LogSegmentRandomAccessEntryReader reader = new BKLogSegmentRandomAccessEntryReader(
- segment,
- lh,
- conf);
- FutureUtils.setValue(openPromise, reader);
- }
- };
- if (segment.isInProgress() && !fence) {
- bk.asyncOpenLedgerNoRecovery(
- segment.getLogSegmentId(),
- BookKeeper.DigestType.CRC32,
- passwd,
- openCallback,
- null);
- } else {
- bk.asyncOpenLedger(
- segment.getLogSegmentId(),
- BookKeeper.DigestType.CRC32,
- passwd,
- openCallback,
- null);
- }
- return openPromise;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
deleted file mode 100644
index 34fe1c3..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
+++ /dev/null
@@ -1,61 +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.logsegment;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-/**
- * Ledger based log segment entry writer.
- */
-public class BKLogSegmentEntryWriter implements LogSegmentEntryWriter {
-
- private final LedgerHandle lh;
-
- public BKLogSegmentEntryWriter(LedgerHandle lh) {
- this.lh = lh;
- }
-
- @VisibleForTesting
- public LedgerHandle getLedgerHandle() {
- return this.lh;
- }
-
- @Override
- public long getLogSegmentId() {
- return lh.getId();
- }
-
- @Override
- public void asyncClose(AsyncCallback.CloseCallback callback, Object ctx) {
- lh.asyncClose(callback, ctx);
- }
-
- @Override
- public void asyncAddEntry(byte[] data, int offset, int length,
- AsyncCallback.AddCallback callback, Object ctx) {
- lh.asyncAddEntry(data, offset, length, callback, ctx);
- }
-
- @Override
- public long size() {
- return lh.getLength();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
deleted file mode 100644
index 9cec80c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
+++ /dev/null
@@ -1,119 +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.logsegment;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-import java.io.IOException;
-import java.util.Enumeration;
-import java.util.List;
-
-/**
- * BookKeeper ledger based random access entry reader.
- */
-class BKLogSegmentRandomAccessEntryReader implements
- LogSegmentRandomAccessEntryReader,
- ReadCallback {
-
- private final long lssn;
- private final long startSequenceId;
- private final boolean envelopeEntries;
- private final boolean deserializeRecordSet;
- // state
- private final LogSegmentMetadata metadata;
- private final LedgerHandle lh;
- private Promise<Void> closePromise = null;
-
- BKLogSegmentRandomAccessEntryReader(LogSegmentMetadata metadata,
- LedgerHandle lh,
- DistributedLogConfiguration conf) {
- this.metadata = metadata;
- this.lssn = metadata.getLogSegmentSequenceNumber();
- this.startSequenceId = metadata.getStartSequenceId();
- this.envelopeEntries = metadata.getEnvelopeEntries();
- this.deserializeRecordSet = conf.getDeserializeRecordSetOnReads();
- this.lh = lh;
- }
-
- @Override
- public long getLastAddConfirmed() {
- return lh.getLastAddConfirmed();
- }
-
- @Override
- public Future<List<Entry.Reader>> readEntries(long startEntryId, long endEntryId) {
- Promise<List<Entry.Reader>> promise = new Promise<List<Entry.Reader>>();
- lh.asyncReadEntries(startEntryId, endEntryId, this, promise);
- return promise;
- }
-
- Entry.Reader processReadEntry(LedgerEntry entry) throws IOException {
- return Entry.newBuilder()
- .setLogSegmentInfo(lssn, startSequenceId)
- .setEntryId(entry.getEntryId())
- .setEnvelopeEntry(envelopeEntries)
- .deserializeRecordSet(deserializeRecordSet)
- .setInputStream(entry.getEntryInputStream())
- .buildReader();
- }
-
- @Override
- public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> entries, Object ctx) {
- Promise<List<Entry.Reader>> promise = (Promise<List<Entry.Reader>>) ctx;
- if (BKException.Code.OK == rc) {
- List<Entry.Reader> entryList = Lists.newArrayList();
- while (entries.hasMoreElements()) {
- try {
- entryList.add(processReadEntry(entries.nextElement()));
- } catch (IOException ioe) {
- FutureUtils.setException(promise, ioe);
- return;
- }
- }
- FutureUtils.setValue(promise, entryList);
- } else {
- FutureUtils.setException(promise,
- new BKTransmitException("Failed to read entries :", rc));
- }
- }
-
- @Override
- public Future<Void> asyncClose() {
- final Promise<Void> closeFuture;
- synchronized (this) {
- if (null != closePromise) {
- return closePromise;
- }
- closeFuture = closePromise = new Promise<Void>();
- }
- BKUtils.closeLedgers(lh).proxyTo(closeFuture);
- return closeFuture;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKUtils.java
deleted file mode 100644
index c71c67e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKUtils.java
+++ /dev/null
@@ -1,72 +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.logsegment;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.function.VoidFunctions;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Futures;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-import java.util.List;
-
-/**
- * BookKeeper Util Functions
- */
-public class BKUtils {
-
- /**
- * Close a ledger <i>lh</i>.
- *
- * @param lh ledger handle
- * @return future represents close result.
- */
- public static Future<Void> closeLedger(LedgerHandle lh) {
- final Promise<Void> closePromise = new Promise<Void>();
- lh.asyncClose(new AsyncCallback.CloseCallback() {
- @Override
- public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
- if (BKException.Code.OK != rc) {
- FutureUtils.setException(closePromise, BKException.create(rc));
- } else {
- FutureUtils.setValue(closePromise, null);
- }
- }
- }, null);
- return closePromise;
- }
-
- /**
- * Close a list of ledgers <i>lhs</i>.
- *
- * @param lhs a list of ledgers
- * @return future represents close results.
- */
- public static Future<Void> closeLedgers(LedgerHandle ... lhs) {
- List<Future<Void>> closeResults = Lists.newArrayListWithExpectedSize(lhs.length);
- for (LedgerHandle lh : lhs) {
- closeResults.add(closeLedger(lh));
- }
- return Futures.collect(closeResults).map(VoidFunctions.LIST_TO_VOID_FUNC);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
deleted file mode 100644
index 3e859fb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
+++ /dev/null
@@ -1,400 +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.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 org.apache.thrift.TException;
-import org.apache.thrift.protocol.TJSONProtocol;
-import org.apache.thrift.transport.TMemoryBuffer;
-import org.apache.thrift.transport.TMemoryInputTransport;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URI;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Configurations for BookKeeper based DL.
- */
-public class BKDLConfig implements DLConfig {
-
- private static final Logger LOG = LoggerFactory.getLogger(BKDLConfig.class);
-
- private static final int BUFFER_SIZE = 4096;
- private static final ConcurrentMap<URI, DLConfig> cachedDLConfigs =
- new ConcurrentHashMap<URI, DLConfig>();
-
- public static void propagateConfiguration(BKDLConfig bkdlConfig, DistributedLogConfiguration dlConf) {
- dlConf.setEncodeRegionIDInLogSegmentMetadata(bkdlConfig.getEncodeRegionID());
- dlConf.setFirstLogSegmentSequenceNumber(bkdlConfig.getFirstLogSegmentSeqNo());
- if (bkdlConfig.isFederatedNamespace()) {
- dlConf.setCreateStreamIfNotExists(false);
- LOG.info("Disabled createIfNotExists for federated namespace.");
- }
- LOG.info("Propagate BKDLConfig to DLConfig : encodeRegionID = {}," +
- " firstLogSegmentSequenceNumber = {}, createStreamIfNotExists = {}, isFederated = {}.",
- new Object[] { dlConf.getEncodeRegionIDInLogSegmentMetadata(),
- dlConf.getFirstLogSegmentSequenceNumber(), dlConf.getCreateStreamIfNotExists(),
- bkdlConfig.isFederatedNamespace() });
- }
-
- public static BKDLConfig resolveDLConfig(ZooKeeperClient zkc, URI uri) throws IOException {
- DLConfig dlConfig = cachedDLConfigs.get(uri);
- if (dlConfig == null) {
- dlConfig = (new ZkMetadataResolver(zkc).resolve(uri)).getDLConfig();
- DLConfig oldDLConfig = cachedDLConfigs.putIfAbsent(uri, dlConfig);
- if (null != oldDLConfig) {
- dlConfig = oldDLConfig;
- }
- }
- assert (dlConfig instanceof BKDLConfig);
- return (BKDLConfig)dlConfig;
- }
-
- @VisibleForTesting
- public static void clearCachedDLConfigs() {
- cachedDLConfigs.clear();
- }
-
- private String bkZkServersForWriter;
- private String bkZkServersForReader;
- private String bkLedgersPath;
- private boolean sanityCheckTxnID = true;
- private boolean encodeRegionID = false;
- private String dlZkServersForWriter;
- private String dlZkServersForReader;
- private String aclRootPath;
- private Long firstLogSegmentSeqNo;
- private boolean isFederatedNamespace = false;
-
- /**
- * Construct a empty config with given <i>uri</i>.
- */
- public BKDLConfig(URI uri) {
- this(BKNamespaceDriver.getZKServersFromDLUri(uri),
- BKNamespaceDriver.getZKServersFromDLUri(uri),
- null, null, null);
- }
-
- /**
- * The caller should make sure both dl and bk use same zookeeper server.
- *
- * @param zkServers
- * zk servers used for both dl and bk.
- * @param ledgersPath
- * ledgers path.
- */
- @VisibleForTesting
- public BKDLConfig(String zkServers, String ledgersPath) {
- this(zkServers, zkServers, zkServers, zkServers, ledgersPath);
- }
-
- public BKDLConfig(String dlZkServersForWriter,
- String dlZkServersForReader,
- String bkZkServersForWriter,
- String bkZkServersForReader,
- String bkLedgersPath) {
- this.dlZkServersForWriter = dlZkServersForWriter;
- this.dlZkServersForReader = dlZkServersForReader;
- this.bkZkServersForWriter = bkZkServersForWriter;
- this.bkZkServersForReader = bkZkServersForReader;
- this.bkLedgersPath = bkLedgersPath;
- }
-
- /**
- * @return zk servers used for bk for writers
- */
- public String getBkZkServersForWriter() {
- return bkZkServersForWriter;
- }
-
- /**
- * @return zk servers used for bk for readers
- */
- public String getBkZkServersForReader() {
- return bkZkServersForReader;
- }
-
- /**
- * @return zk servers used for dl for writers
- */
- public String getDlZkServersForWriter() {
- return dlZkServersForWriter;
- }
-
- /**
- * @return zk servers used for dl for readers
- */
- public String getDlZkServersForReader() {
- return dlZkServersForReader;
- }
-
- /**
- * @return ledgers path for bk
- */
- public String getBkLedgersPath() {
- return bkLedgersPath;
- }
-
- /**
- * Enable/Disable sanity check txn id.
- *
- * @param enabled
- * flag to enable/disable sanity check txn id.
- * @return bk dl config.
- */
- public BKDLConfig setSanityCheckTxnID(boolean enabled) {
- this.sanityCheckTxnID = enabled;
- return this;
- }
-
- /**
- * @return flag to sanity check highest txn id.
- */
- public boolean getSanityCheckTxnID() {
- return sanityCheckTxnID;
- }
-
- /**
- * Enable/Disable encode region id.
- *
- * @param enabled
- * flag to enable/disable encoding region id.
- * @return bk dl config
- */
- public BKDLConfig setEncodeRegionID(boolean enabled) {
- this.encodeRegionID = enabled;
- return this;
- }
-
- /**
- * @return flag to encode region id.
- */
- public boolean getEncodeRegionID() {
- return encodeRegionID;
- }
-
- /**
- * Set the root path of zk based ACL manager.
- *
- * @param aclRootPath
- * root path of zk based ACL manager.
- * @return bk dl config
- */
- public BKDLConfig setACLRootPath(String aclRootPath) {
- this.aclRootPath = aclRootPath;
- return this;
- }
-
- /**
- * Get the root path of zk based ACL manager.
- *
- * @return root path of zk based ACL manager.
- */
- public String getACLRootPath() {
- return aclRootPath;
- }
-
- /**
- * Set the value at which ledger sequence number should start for streams that are being
- * upgraded and did not have ledger sequence number to start with or for newly created
- * streams
- *
- * @param firstLogSegmentSeqNo first ledger sequence number
- * @return bk dl config
- */
- public BKDLConfig setFirstLogSegmentSeqNo(long firstLogSegmentSeqNo) {
- this.firstLogSegmentSeqNo = firstLogSegmentSeqNo;
- return this;
- }
-
- /**
- * Get the value at which ledger sequence number should start for streams that are being
- * upgraded and did not have ledger sequence number to start with or for newly created
- * streams
- *
- * @return first ledger sequence number
- */
- public Long getFirstLogSegmentSeqNo() {
- if (null == firstLogSegmentSeqNo) {
- return DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO;
- }
- return firstLogSegmentSeqNo;
- }
-
- /**
- * Set the namespace to federated <i>isFederatedNamespace</i>.
- *
- * @param isFederatedNamespace
- * is the namespace federated?
- * @return bk dl config
- */
- public BKDLConfig setFederatedNamespace(boolean isFederatedNamespace) {
- this.isFederatedNamespace = isFederatedNamespace;
- return this;
- }
-
- /**
- * Whether the namespace is federated namespace
- *
- * @return true if the namespace is a federated namespace. otherwise false.
- */
- public boolean isFederatedNamespace() {
- return this.isFederatedNamespace;
- }
-
- @Override
- public int hashCode() {
- return Objects.hashCode(bkZkServersForWriter, bkZkServersForReader,
- dlZkServersForWriter, dlZkServersForReader,
- bkLedgersPath);
- }
-
- @Override
- public boolean equals(Object o) {
- if (!(o instanceof BKDLConfig)) {
- return false;
- }
- BKDLConfig another = (BKDLConfig) o;
- return Objects.equal(bkZkServersForWriter, another.bkZkServersForWriter) &&
- Objects.equal(bkZkServersForReader, another.bkZkServersForReader) &&
- Objects.equal(dlZkServersForWriter, another.dlZkServersForWriter) &&
- Objects.equal(dlZkServersForReader, another.dlZkServersForReader) &&
- Objects.equal(bkLedgersPath, another.bkLedgersPath) &&
- sanityCheckTxnID == another.sanityCheckTxnID &&
- encodeRegionID == another.encodeRegionID &&
- Objects.equal(aclRootPath, another.aclRootPath) &&
- Objects.equal(firstLogSegmentSeqNo, another.firstLogSegmentSeqNo) &&
- Objects.equal(isFederatedNamespace, another.isFederatedNamespace);
-
- }
-
- @Override
- public String toString() {
- return serialize();
- }
-
- @Override
- public String serialize() {
- BKDLConfigFormat configFormat = new BKDLConfigFormat();
- if (null != bkZkServersForWriter) {
- configFormat.setBkZkServers(bkZkServersForWriter);
- }
- if (null != bkZkServersForReader) {
- configFormat.setBkZkServersForReader(bkZkServersForReader);
- }
- if (null != dlZkServersForWriter) {
- configFormat.setDlZkServersForWriter(dlZkServersForWriter);
- }
- if (null != dlZkServersForReader) {
- configFormat.setDlZkServersForReader(dlZkServersForReader);
- }
- if (null != bkLedgersPath) {
- configFormat.setBkLedgersPath(bkLedgersPath);
- }
- configFormat.setSanityCheckTxnID(sanityCheckTxnID);
- configFormat.setEncodeRegionID(encodeRegionID);
- if (null != aclRootPath) {
- configFormat.setAclRootPath(aclRootPath);
- }
- if (null != firstLogSegmentSeqNo) {
- configFormat.setFirstLogSegmentSeqNo(firstLogSegmentSeqNo);
- }
- if (isFederatedNamespace) {
- configFormat.setFederatedNamespace(true);
- }
- return serialize(configFormat);
- }
-
- String serialize(BKDLConfigFormat configFormat) {
- TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
- TJSONProtocol protocol = new TJSONProtocol(transport);
- try {
- configFormat.write(protocol);
- transport.flush();
- return transport.toString("UTF-8");
- } catch (TException e) {
- throw new RuntimeException("Failed to serialize BKDLConfig : ", e);
- } catch (UnsupportedEncodingException e) {
- throw new RuntimeException("Failed to serialize BKDLConfig : ", e);
- }
- }
-
- @Override
- public void deserialize(byte[] data) throws IOException {
- BKDLConfigFormat configFormat = new BKDLConfigFormat();
- TMemoryInputTransport transport = new TMemoryInputTransport(data);
- TJSONProtocol protocol = new TJSONProtocol(transport);
- try {
- configFormat.read(protocol);
- } catch (TException e) {
- throw new IOException("Failed to deserialize data '" +
- new String(data, UTF_8) + "' : ", e);
- }
- // bookkeeper cluster settings
- if (configFormat.isSetBkZkServers()) {
- bkZkServersForWriter = configFormat.getBkZkServers();
- }
- if (configFormat.isSetBkZkServersForReader()) {
- bkZkServersForReader = configFormat.getBkZkServersForReader();
- } else {
- bkZkServersForReader = bkZkServersForWriter;
- }
- if (configFormat.isSetBkLedgersPath()) {
- bkLedgersPath = configFormat.getBkLedgersPath();
- }
- // dl zookeeper cluster settings
- if (configFormat.isSetDlZkServersForWriter()) {
- dlZkServersForWriter = configFormat.getDlZkServersForWriter();
- }
- if (configFormat.isSetDlZkServersForReader()) {
- dlZkServersForReader = configFormat.getDlZkServersForReader();
- } else {
- dlZkServersForReader = dlZkServersForWriter;
- }
- // dl settings
- sanityCheckTxnID = !configFormat.isSetSanityCheckTxnID() || configFormat.isSanityCheckTxnID();
- encodeRegionID = configFormat.isSetEncodeRegionID() && configFormat.isEncodeRegionID();
- if (configFormat.isSetAclRootPath()) {
- aclRootPath = configFormat.getAclRootPath();
- }
-
- if (configFormat.isSetFirstLogSegmentSeqNo()) {
- firstLogSegmentSeqNo = configFormat.getFirstLogSegmentSeqNo();
- }
- isFederatedNamespace = configFormat.isSetFederatedNamespace() && configFormat.isFederatedNamespace();
-
- // Validate the settings
- if (null == bkZkServersForWriter || null == bkZkServersForReader || null == bkLedgersPath ||
- null == dlZkServersForWriter || null == dlZkServersForReader) {
- throw new IOException("Missing zk/bk settings in BKDL Config : " + new String(data, UTF_8));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
deleted file mode 100644
index c76a5a5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
+++ /dev/null
@@ -1,633 +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.metadata;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.exceptions.LockCancelledException;
-import com.twitter.distributedlog.exceptions.LogExistsException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.impl.ZKLogSegmentMetadataStore;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.lock.SessionLockFactory;
-import com.twitter.distributedlog.lock.ZKDistributedLock;
-import com.twitter.distributedlog.lock.ZKSessionLockFactory;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.metadata.LogMetadataForReader;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.distributedlog.zk.LimitedPermitManager;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitManager;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.zk.ZKTransaction;
-import com.twitter.util.ExceptionalFunction;
-import com.twitter.util.ExceptionalFunction0;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-import org.apache.zookeeper.ZKUtil;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.common.PathUtils;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static com.twitter.distributedlog.metadata.LogMetadata.*;
-
-/**
- * zookeeper based {@link LogStreamMetadataStore}
- */
-public class ZKLogStreamMetadataStore implements LogStreamMetadataStore {
-
- private final static Logger LOG = LoggerFactory.getLogger(ZKLogStreamMetadataStore.class);
-
- private final String clientId;
- private final DistributedLogConfiguration conf;
- private final ZooKeeperClient zooKeeperClient;
- private final OrderedScheduler scheduler;
- private final StatsLogger statsLogger;
- private final LogSegmentMetadataStore logSegmentStore;
- private final LimitedPermitManager permitManager;
- // lock
- private SessionLockFactory lockFactory;
- private OrderedScheduler lockStateExecutor;
-
- public ZKLogStreamMetadataStore(String clientId,
- DistributedLogConfiguration conf,
- ZooKeeperClient zkc,
- OrderedScheduler scheduler,
- StatsLogger statsLogger) {
- this.clientId = clientId;
- this.conf = conf;
- this.zooKeeperClient = zkc;
- this.scheduler = scheduler;
- this.statsLogger = statsLogger;
- // create the log segment metadata store and the permit manager (used for log segment rolling)
- this.logSegmentStore = new ZKLogSegmentMetadataStore(conf, zooKeeperClient, scheduler);
- this.permitManager = new LimitedPermitManager(
- conf.getLogSegmentRollingConcurrency(),
- 1,
- TimeUnit.MINUTES,
- scheduler);
- this.zooKeeperClient.register(permitManager);
- }
-
- private synchronized OrderedScheduler getLockStateExecutor(boolean createIfNull) {
- if (createIfNull && null == lockStateExecutor) {
- StatsLogger lockStateStatsLogger = statsLogger.scope("lock_scheduler");
- lockStateExecutor = OrderedScheduler.newBuilder()
- .name("DLM-LockState")
- .corePoolSize(conf.getNumLockStateThreads())
- .statsLogger(lockStateStatsLogger)
- .perExecutorStatsLogger(lockStateStatsLogger)
- .traceTaskExecution(conf.getEnableTaskExecutionStats())
- .traceTaskExecutionWarnTimeUs(conf.getTaskExecutionWarnTimeMicros())
- .build();
- }
- return lockStateExecutor;
- }
-
- private synchronized SessionLockFactory getLockFactory(boolean createIfNull) {
- if (createIfNull && null == lockFactory) {
- lockFactory = new ZKSessionLockFactory(
- zooKeeperClient,
- clientId,
- getLockStateExecutor(createIfNull),
- conf.getZKNumRetries(),
- conf.getLockTimeoutMilliSeconds(),
- conf.getZKRetryBackoffStartMillis(),
- statsLogger);
- }
- return lockFactory;
- }
-
- @Override
- public void close() throws IOException {
- this.zooKeeperClient.unregister(permitManager);
- this.permitManager.close();
- this.logSegmentStore.close();
- SchedulerUtils.shutdownScheduler(
- getLockStateExecutor(false),
- conf.getSchedulerShutdownTimeoutMs(),
- TimeUnit.MILLISECONDS);
- }
-
- @Override
- public LogSegmentMetadataStore getLogSegmentMetadataStore() {
- return logSegmentStore;
- }
-
- @Override
- public PermitManager getPermitManager() {
- return this.permitManager;
- }
-
- @Override
- public Transaction<Object> newTransaction() {
- return new ZKTransaction(zooKeeperClient);
- }
-
- @Override
- public Future<Void> logExists(URI uri, final String logName) {
- final String logSegmentsPath = LogMetadata.getLogSegmentsPath(
- uri, logName, conf.getUnpartitionedStreamName());
- final Promise<Void> promise = new Promise<Void>();
- try {
- final ZooKeeper zk = zooKeeperClient.get();
- zk.sync(logSegmentsPath, new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int syncRc, String path, Object syncCtx) {
- if (KeeperException.Code.NONODE.intValue() == syncRc) {
- promise.setException(new LogNotFoundException(
- String.format("Log %s does not exist or has been deleted", logName)));
- return;
- } else if (KeeperException.Code.OK.intValue() != syncRc){
- promise.setException(new ZKException("Error on checking log existence for " + logName,
- KeeperException.create(KeeperException.Code.get(syncRc))));
- return;
- }
- zk.exists(logSegmentsPath, false, new AsyncCallback.StatCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, Stat stat) {
- if (KeeperException.Code.OK.intValue() == rc) {
- promise.setValue(null);
- } else if (KeeperException.Code.NONODE.intValue() == rc) {
- promise.setException(new LogNotFoundException(
- String.format("Log %s does not exist or has been deleted", logName)));
- } else {
- promise.setException(new ZKException("Error on checking log existence for " + logName,
- KeeperException.create(KeeperException.Code.get(rc))));
- }
- }
- }, null);
- }
- }, null);
-
- } catch (InterruptedException ie) {
- LOG.error("Interrupted while reading {}", logSegmentsPath, ie);
- promise.setException(new DLInterruptedException("Interrupted while checking "
- + logSegmentsPath, ie));
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- }
- return promise;
- }
-
- //
- // Create Write Lock
- //
-
- @Override
- public DistributedLock createWriteLock(LogMetadataForWriter metadata) {
- return new ZKDistributedLock(
- getLockStateExecutor(true),
- getLockFactory(true),
- metadata.getLockPath(),
- conf.getLockTimeoutMilliSeconds(),
- statsLogger);
- }
-
- //
- // Create Read Lock
- //
-
- private Future<Void> ensureReadLockPathExist(final LogMetadata logMetadata,
- final String readLockPath) {
- final Promise<Void> promise = new Promise<Void>();
- promise.setInterruptHandler(new com.twitter.util.Function<Throwable, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Throwable t) {
- FutureUtils.setException(promise, new LockCancelledException(readLockPath,
- "Could not ensure read lock path", t));
- return null;
- }
- });
- Optional<String> parentPathShouldNotCreate = Optional.of(logMetadata.getLogRootPath());
- Utils.zkAsyncCreateFullPathOptimisticRecursive(zooKeeperClient, readLockPath, parentPathShouldNotCreate,
- new byte[0], zooKeeperClient.getDefaultACL(), CreateMode.PERSISTENT,
- new org.apache.zookeeper.AsyncCallback.StringCallback() {
- @Override
- public void processResult(final int rc, final String path, Object ctx, String name) {
- if (KeeperException.Code.NONODE.intValue() == rc) {
- FutureUtils.setException(promise, new LogNotFoundException(
- String.format("Log %s does not exist or has been deleted",
- logMetadata.getFullyQualifiedName())));
- } else if (KeeperException.Code.OK.intValue() == rc) {
- FutureUtils.setValue(promise, null);
- LOG.trace("Created path {}.", path);
- } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
- FutureUtils.setValue(promise, null);
- LOG.trace("Path {} is already existed.", path);
- } else if (DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE == rc) {
- FutureUtils.setException(promise, new ZooKeeperClient.ZooKeeperConnectionException(path));
- } else if (DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE == rc) {
- FutureUtils.setException(promise, new DLInterruptedException(path));
- } else {
- FutureUtils.setException(promise, KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
- }, null);
- return promise;
- }
-
- @Override
- public Future<DistributedLock> createReadLock(final LogMetadataForReader metadata,
- Optional<String> readerId) {
- final String readLockPath = metadata.getReadLockPath(readerId);
- return ensureReadLockPathExist(metadata, readLockPath).flatMap(
- new ExceptionalFunction<Void, Future<DistributedLock>>() {
- @Override
- public Future<DistributedLock> applyE(Void value) throws Throwable {
- // Unfortunately this has a blocking call which we should not execute on the
- // ZK completion thread
- return scheduler.apply(new ExceptionalFunction0<DistributedLock>() {
- @Override
- public DistributedLock applyE() throws Throwable {
- return new ZKDistributedLock(
- getLockStateExecutor(true),
- getLockFactory(true),
- readLockPath,
- conf.getLockTimeoutMilliSeconds(),
- statsLogger.scope("read_lock"));
- }
- });
- }
- });
- }
-
- //
- // Create Log
- //
-
- static class MetadataIndex {
- static final int LOG_ROOT_PARENT = 0;
- static final int LOG_ROOT = 1;
- static final int MAX_TXID = 2;
- static final int VERSION = 3;
- static final int LOCK = 4;
- static final int READ_LOCK = 5;
- static final int LOGSEGMENTS = 6;
- static final int ALLOCATION = 7;
- }
-
- static int bytesToInt(byte[] b) {
- assert b.length >= 4;
- return b[0] << 24 | b[1] << 16 | b[2] << 8 | b[3];
- }
-
- static byte[] intToBytes(int i) {
- return new byte[]{
- (byte) (i >> 24),
- (byte) (i >> 16),
- (byte) (i >> 8),
- (byte) (i)};
- }
-
- static Future<List<Versioned<byte[]>>> checkLogMetadataPaths(ZooKeeper zk,
- String logRootPath,
- boolean ownAllocator) {
- // Note re. persistent lock state initialization: the read lock persistent state (path) is
- // initialized here but only used in the read handler. The reason is its more convenient and
- // less error prone to manage all stream structure in one place.
- final String logRootParentPath = new File(logRootPath).getParent();
- final String logSegmentsPath = logRootPath + LOGSEGMENTS_PATH;
- final String maxTxIdPath = logRootPath + MAX_TXID_PATH;
- final String lockPath = logRootPath + LOCK_PATH;
- final String readLockPath = logRootPath + READ_LOCK_PATH;
- final String versionPath = logRootPath + VERSION_PATH;
- final String allocationPath = logRootPath + ALLOCATION_PATH;
-
- int numPaths = ownAllocator ? MetadataIndex.ALLOCATION + 1 : MetadataIndex.LOGSEGMENTS + 1;
- List<Future<Versioned<byte[]>>> checkFutures = Lists.newArrayListWithExpectedSize(numPaths);
- checkFutures.add(Utils.zkGetData(zk, logRootParentPath, false));
- checkFutures.add(Utils.zkGetData(zk, logRootPath, false));
- checkFutures.add(Utils.zkGetData(zk, maxTxIdPath, false));
- checkFutures.add(Utils.zkGetData(zk, versionPath, false));
- checkFutures.add(Utils.zkGetData(zk, lockPath, false));
- checkFutures.add(Utils.zkGetData(zk, readLockPath, false));
- checkFutures.add(Utils.zkGetData(zk, logSegmentsPath, false));
- if (ownAllocator) {
- checkFutures.add(Utils.zkGetData(zk, allocationPath, false));
- }
-
- return Future.collect(checkFutures);
- }
-
- static boolean pathExists(Versioned<byte[]> metadata) {
- return null != metadata.getValue() && null != metadata.getVersion();
- }
-
- static void ensureMetadataExist(Versioned<byte[]> metadata) {
- Preconditions.checkNotNull(metadata.getValue());
- Preconditions.checkNotNull(metadata.getVersion());
- }
-
- static void createMissingMetadata(final ZooKeeper zk,
- final String logRootPath,
- final List<Versioned<byte[]>> metadatas,
- final List<ACL> acl,
- final boolean ownAllocator,
- final boolean createIfNotExists,
- final Promise<List<Versioned<byte[]>>> promise) {
- final List<byte[]> pathsToCreate = Lists.newArrayListWithExpectedSize(metadatas.size());
- final List<Op> zkOps = Lists.newArrayListWithExpectedSize(metadatas.size());
- CreateMode createMode = CreateMode.PERSISTENT;
-
- // log root parent path
- if (pathExists(metadatas.get(MetadataIndex.LOG_ROOT_PARENT))) {
- pathsToCreate.add(null);
- } else {
- String logRootParentPath = new File(logRootPath).getParent();
- pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
- zkOps.add(Op.create(logRootParentPath, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
- }
-
- // log root path
- if (pathExists(metadatas.get(MetadataIndex.LOG_ROOT))) {
- pathsToCreate.add(null);
- } else {
- pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
- zkOps.add(Op.create(logRootPath, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
- }
-
- // max id
- if (pathExists(metadatas.get(MetadataIndex.MAX_TXID))) {
- pathsToCreate.add(null);
- } else {
- byte[] zeroTxnIdData = DLUtils.serializeTransactionId(0L);
- pathsToCreate.add(zeroTxnIdData);
- zkOps.add(Op.create(logRootPath + MAX_TXID_PATH, zeroTxnIdData, acl, createMode));
- }
- // version
- if (pathExists(metadatas.get(MetadataIndex.VERSION))) {
- pathsToCreate.add(null);
- } else {
- byte[] versionData = intToBytes(LAYOUT_VERSION);
- pathsToCreate.add(versionData);
- zkOps.add(Op.create(logRootPath + VERSION_PATH, versionData, acl, createMode));
- }
- // lock path
- if (pathExists(metadatas.get(MetadataIndex.LOCK))) {
- pathsToCreate.add(null);
- } else {
- pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
- zkOps.add(Op.create(logRootPath + LOCK_PATH, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
- }
- // read lock path
- if (pathExists(metadatas.get(MetadataIndex.READ_LOCK))) {
- pathsToCreate.add(null);
- } else {
- pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
- zkOps.add(Op.create(logRootPath + READ_LOCK_PATH, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
- }
- // log segments path
- if (pathExists(metadatas.get(MetadataIndex.LOGSEGMENTS))) {
- pathsToCreate.add(null);
- } else {
- byte[] logSegmentsData = DLUtils.serializeLogSegmentSequenceNumber(
- DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO);
- pathsToCreate.add(logSegmentsData);
- zkOps.add(Op.create(logRootPath + LOGSEGMENTS_PATH, logSegmentsData, acl, createMode));
- }
- // allocation path
- if (ownAllocator) {
- if (pathExists(metadatas.get(MetadataIndex.ALLOCATION))) {
- pathsToCreate.add(null);
- } else {
- pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
- zkOps.add(Op.create(logRootPath + ALLOCATION_PATH,
- DistributedLogConstants.EMPTY_BYTES, acl, createMode));
- }
- }
- if (zkOps.isEmpty()) {
- // nothing missed
- promise.setValue(metadatas);
- return;
- }
- if (!createIfNotExists) {
- promise.setException(new LogNotFoundException("Log " + logRootPath + " not found"));
- return;
- }
-
- zk.multi(zkOps, new AsyncCallback.MultiCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, List<OpResult> resultList) {
- if (KeeperException.Code.OK.intValue() == rc) {
- List<Versioned<byte[]>> finalMetadatas =
- Lists.newArrayListWithExpectedSize(metadatas.size());
- for (int i = 0; i < pathsToCreate.size(); i++) {
- byte[] dataCreated = pathsToCreate.get(i);
- if (null == dataCreated) {
- finalMetadatas.add(metadatas.get(i));
- } else {
- finalMetadatas.add(new Versioned<byte[]>(dataCreated, new ZkVersion(0)));
- }
- }
- promise.setValue(finalMetadatas);
- } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
- promise.setException(new LogExistsException("Someone just created log "
- + logRootPath));
- } else {
- if (LOG.isDebugEnabled()) {
- StringBuilder builder = new StringBuilder();
- for (OpResult result : resultList) {
- if (result instanceof OpResult.ErrorResult) {
- OpResult.ErrorResult errorResult = (OpResult.ErrorResult) result;
- builder.append(errorResult.getErr()).append(",");
- } else {
- builder.append(0).append(",");
- }
- }
- String resultCodeList = builder.substring(0, builder.length() - 1);
- LOG.debug("Failed to create log, full rc list = {}", resultCodeList);
- }
-
- promise.setException(new ZKException("Failed to create log " + logRootPath,
- KeeperException.Code.get(rc)));
- }
- }
- }, null);
- }
-
- static LogMetadataForWriter processLogMetadatas(URI uri,
- String logName,
- String logIdentifier,
- List<Versioned<byte[]>> metadatas,
- boolean ownAllocator)
- throws UnexpectedException {
- try {
- // max id
- Versioned<byte[]> maxTxnIdData = metadatas.get(MetadataIndex.MAX_TXID);
- ensureMetadataExist(maxTxnIdData);
- // version
- Versioned<byte[]> versionData = metadatas.get(MetadataIndex.VERSION);
- ensureMetadataExist(maxTxnIdData);
- Preconditions.checkArgument(LAYOUT_VERSION == bytesToInt(versionData.getValue()));
- // lock path
- ensureMetadataExist(metadatas.get(MetadataIndex.LOCK));
- // read lock path
- ensureMetadataExist(metadatas.get(MetadataIndex.READ_LOCK));
- // max lssn
- Versioned<byte[]> maxLSSNData = metadatas.get(MetadataIndex.LOGSEGMENTS);
- ensureMetadataExist(maxLSSNData);
- try {
- DLUtils.deserializeLogSegmentSequenceNumber(maxLSSNData.getValue());
- } catch (NumberFormatException nfe) {
- throw new UnexpectedException("Invalid max sequence number found in log " + logName, nfe);
- }
- // allocation path
- Versioned<byte[]> allocationData;
- if (ownAllocator) {
- allocationData = metadatas.get(MetadataIndex.ALLOCATION);
- ensureMetadataExist(allocationData);
- } else {
- allocationData = new Versioned<byte[]>(null, null);
- }
- return new LogMetadataForWriter(uri, logName, logIdentifier,
- maxLSSNData, maxTxnIdData, allocationData);
- } catch (IllegalArgumentException iae) {
- throw new UnexpectedException("Invalid log " + logName, iae);
- } catch (NullPointerException npe) {
- throw new UnexpectedException("Invalid log " + logName, npe);
- }
- }
-
- static Future<LogMetadataForWriter> getLog(final URI uri,
- final String logName,
- final String logIdentifier,
- final ZooKeeperClient zooKeeperClient,
- final boolean ownAllocator,
- final boolean createIfNotExists) {
- final String logRootPath = LogMetadata.getLogRootPath(uri, logName, logIdentifier);
- try {
- PathUtils.validatePath(logRootPath);
- } catch (IllegalArgumentException e) {
- LOG.error("Illegal path value {} for stream {}", new Object[]{logRootPath, logName, e});
- return Future.exception(new InvalidStreamNameException(logName, "Log name is invalid"));
- }
-
- try {
- final ZooKeeper zk = zooKeeperClient.get();
- return checkLogMetadataPaths(zk, logRootPath, ownAllocator)
- .flatMap(new AbstractFunction1<List<Versioned<byte[]>>, Future<List<Versioned<byte[]>>>>() {
- @Override
- public Future<List<Versioned<byte[]>>> apply(List<Versioned<byte[]>> metadatas) {
- Promise<List<Versioned<byte[]>>> promise =
- new Promise<List<Versioned<byte[]>>>();
- createMissingMetadata(zk, logRootPath, metadatas, zooKeeperClient.getDefaultACL(),
- ownAllocator, createIfNotExists, promise);
- return promise;
- }
- }).map(new ExceptionalFunction<List<Versioned<byte[]>>, LogMetadataForWriter>() {
- @Override
- public LogMetadataForWriter applyE(List<Versioned<byte[]>> metadatas) throws DLException {
- return processLogMetadatas(
- uri,
- logName,
- logIdentifier,
- metadatas,
- ownAllocator);
- }
- });
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- return Future.exception(new ZKException("Encountered zookeeper connection issue on creating log " + logName,
- KeeperException.Code.CONNECTIONLOSS));
- } catch (InterruptedException e) {
- return Future.exception(new DLInterruptedException("Interrupted on creating log " + logName, e));
- }
- }
-
- @Override
- public Future<LogMetadataForWriter> getLog(final URI uri,
- final String logName,
- final boolean ownAllocator,
- final boolean createIfNotExists) {
- return getLog(
- uri,
- logName,
- conf.getUnpartitionedStreamName(),
- zooKeeperClient,
- ownAllocator,
- createIfNotExists);
- }
-
- //
- // Delete Log
- //
-
- @Override
- public Future<Void> deleteLog(URI uri, final String logName) {
- final Promise<Void> promise = new Promise<Void>();
- try {
- String streamPath = LogMetadata.getLogStreamPath(uri, logName);
- ZKUtil.deleteRecursive(zooKeeperClient.get(), streamPath, new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx) {
- if (KeeperException.Code.OK.intValue() != rc) {
- FutureUtils.setException(promise,
- new ZKException("Encountered zookeeper issue on deleting log stream "
- + logName, KeeperException.Code.get(rc)));
- return;
- }
- FutureUtils.setValue(promise, null);
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- FutureUtils.setException(promise, new ZKException("Encountered zookeeper issue on deleting log stream "
- + logName, KeeperException.Code.CONNECTIONLOSS));
- } catch (InterruptedException e) {
- FutureUtils.setException(promise, new DLInterruptedException("Interrupted while deleting log stream "
- + logName));
- } catch (KeeperException e) {
- FutureUtils.setException(promise, new ZKException("Encountered zookeeper issue on deleting log stream "
- + logName, e));
- }
- return promise;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
deleted file mode 100644
index 6b7a231..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
+++ /dev/null
@@ -1,72 +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.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;
-import org.apache.zookeeper.data.Stat;
-
-import java.io.IOException;
-import java.net.URI;
-
-public class ZkMetadataResolver implements MetadataResolver {
-
- private final ZooKeeperClient zkc;
-
- public ZkMetadataResolver(ZooKeeperClient zkc) {
- this.zkc = zkc;
- }
-
- @Override
- public DLMetadata resolve(URI uri) throws IOException {
- String dlPath = uri.getPath();
- PathUtils.validatePath(dlPath);
- // Normal case the dl metadata is stored in the last segment
- // so lookup last segment first.
- String[] parts = StringUtils.split(dlPath, '/');
- if (null == parts || 0 == parts.length) {
- throw new IOException("Invalid dlPath to resolve dl metadata : " + dlPath);
- }
- for (int i = parts.length; i >= 0; i--) {
- String pathToResolve = String.format("/%s", StringUtils.join(parts, '/', 0, i));
- byte[] data;
- try {
- data = zkc.get().getData(pathToResolve, false, new Stat());
- } catch (KeeperException.NoNodeException nne) {
- continue;
- } catch (KeeperException ke) {
- throw new IOException("Fail to resolve dl path : " + pathToResolve);
- } catch (InterruptedException ie) {
- throw new IOException("Interrupted when resolving dl path : " + pathToResolve);
- }
- if (null == data || data.length == 0) {
- continue;
- }
- try {
- return DLMetadata.deserialize(uri, data);
- } catch (IOException ie) {
- throw new IOException("Failed to deserialize uri : " + uri);
- }
- }
- throw new IOException("No bkdl config bound under dl path : " + dlPath);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/package-info.java
deleted file mode 100644
index 7c5c2e4..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * The BookKeeper Based DistributedLog Implementation.
- */
-package com.twitter.distributedlog.impl;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
deleted file mode 100644
index b067ee9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
+++ /dev/null
@@ -1,121 +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.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;
-
-import com.google.common.base.Charsets;
-
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-
-public class ZKSubscriptionStateStore implements SubscriptionStateStore {
-
- static final Logger logger = LoggerFactory.getLogger(ZKSubscriptionStateStore.class);
-
- private final ZooKeeperClient zooKeeperClient;
- private final String zkPath;
- private AtomicReference<DLSN> lastCommittedPosition = new AtomicReference<DLSN>(null);
-
- public ZKSubscriptionStateStore(ZooKeeperClient zooKeeperClient, String zkPath) {
- this.zooKeeperClient = zooKeeperClient;
- this.zkPath = zkPath;
- }
-
- @Override
- public void close() throws IOException {
- }
-
- /**
- * Get the last committed position stored for this subscription
- */
- @Override
- public Future<DLSN> getLastCommitPosition() {
- if (null != lastCommittedPosition.get()) {
- return Future.value(lastCommittedPosition.get());
- } else {
- return getLastCommitPositionFromZK();
- }
- }
-
- Future<DLSN> getLastCommitPositionFromZK() {
- final Promise<DLSN> result = new Promise<DLSN>();
- try {
- logger.debug("Reading last commit position from path {}", zkPath);
- zooKeeperClient.get().getData(zkPath, false, new AsyncCallback.DataCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
- logger.debug("Read last commit position from path {}: rc = {}", zkPath, rc);
- if (KeeperException.Code.NONODE.intValue() == rc) {
- result.setValue(DLSN.NonInclusiveLowerBound);
- } else if (KeeperException.Code.OK.intValue() != rc) {
- result.setException(KeeperException.create(KeeperException.Code.get(rc), path));
- } else {
- try {
- DLSN dlsn = DLSN.deserialize(new String(data, Charsets.UTF_8));
- result.setValue(dlsn);
- } catch (Exception t) {
- logger.warn("Invalid last commit position found from path {}", zkPath, t);
- // invalid dlsn recorded in subscription state store
- result.setValue(DLSN.NonInclusiveLowerBound);
- }
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
- result.setException(zkce);
- } catch (InterruptedException ie) {
- result.setException(new DLInterruptedException("getLastCommitPosition was interrupted", ie));
- }
- return result;
- }
-
- /**
- * Advances the position associated with the subscriber
- *
- * @param newPosition - new commit position
- */
- @Override
- public Future<BoxedUnit> advanceCommitPosition(DLSN newPosition) {
- if (null == lastCommittedPosition.get() ||
- (newPosition.compareTo(lastCommittedPosition.get()) > 0)) {
- lastCommittedPosition.set(newPosition);
- return Utils.zkAsyncCreateFullPathOptimisticAndSetData(zooKeeperClient,
- zkPath, newPosition.serialize().getBytes(Charsets.UTF_8),
- zooKeeperClient.getDefaultACL(),
- CreateMode.PERSISTENT);
- } else {
- return Future.Done();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
deleted file mode 100644
index 17ba943..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
+++ /dev/null
@@ -1,164 +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.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.Future;
-import com.twitter.util.Promise;
-
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * ZooKeeper Based Subscriptions Store.
- */
-public class ZKSubscriptionsStore implements SubscriptionsStore {
-
- private final ZooKeeperClient zkc;
- private final String zkPath;
- private final ConcurrentMap<String, ZKSubscriptionStateStore> subscribers =
- new ConcurrentHashMap<String, ZKSubscriptionStateStore>();
-
- public ZKSubscriptionsStore(ZooKeeperClient zkc, String zkPath) {
- this.zkc = zkc;
- this.zkPath = zkPath;
- }
-
- private ZKSubscriptionStateStore getSubscriber(String subscriberId) {
- ZKSubscriptionStateStore ss = subscribers.get(subscriberId);
- if (ss == null) {
- ZKSubscriptionStateStore newSS = new ZKSubscriptionStateStore(zkc,
- getSubscriberZKPath(subscriberId));
- ZKSubscriptionStateStore oldSS = subscribers.putIfAbsent(subscriberId, newSS);
- if (oldSS == null) {
- ss = newSS;
- } else {
- try {
- newSS.close();
- } catch (IOException e) {
- // ignore the exception
- }
- ss = oldSS;
- }
- }
- return ss;
- }
-
- private String getSubscriberZKPath(String subscriberId) {
- return String.format("%s/%s", zkPath, subscriberId);
- }
-
- @Override
- public Future<DLSN> getLastCommitPosition(String subscriberId) {
- return getSubscriber(subscriberId).getLastCommitPosition();
- }
-
- @Override
- public Future<Map<String, DLSN>> getLastCommitPositions() {
- final Promise<Map<String, DLSN>> result = new Promise<Map<String, DLSN>>();
- try {
- this.zkc.get().getChildren(this.zkPath, false, new AsyncCallback.Children2Callback() {
- @Override
- public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
- if (KeeperException.Code.NONODE.intValue() == rc) {
- result.setValue(new HashMap<String, DLSN>());
- } else if (KeeperException.Code.OK.intValue() != rc) {
- result.setException(KeeperException.create(KeeperException.Code.get(rc), path));
- } else {
- getLastCommitPositions(result, children);
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
- result.setException(zkce);
- } catch (InterruptedException ie) {
- result.setException(new DLInterruptedException("getLastCommitPositions was interrupted", ie));
- }
- return result;
- }
-
- private void getLastCommitPositions(final Promise<Map<String, DLSN>> result,
- List<String> subscribers) {
- List<Future<Pair<String, DLSN>>> futures =
- new ArrayList<Future<Pair<String, DLSN>>>(subscribers.size());
- for (String s : subscribers) {
- final String subscriber = s;
- Future<Pair<String, DLSN>> future =
- // Get the last commit position from zookeeper
- getSubscriber(subscriber).getLastCommitPositionFromZK().map(
- new AbstractFunction1<DLSN, Pair<String, DLSN>>() {
- @Override
- public Pair<String, DLSN> apply(DLSN dlsn) {
- return Pair.of(subscriber, dlsn);
- }
- });
- futures.add(future);
- }
- Future.collect(futures).foreach(
- new AbstractFunction1<List<Pair<String, DLSN>>, BoxedUnit>() {
- @Override
- public BoxedUnit apply(List<Pair<String, DLSN>> subscriptions) {
- Map<String, DLSN> subscriptionMap = new HashMap<String, DLSN>();
- for (Pair<String, DLSN> pair : subscriptions) {
- subscriptionMap.put(pair.getLeft(), pair.getRight());
- }
- result.setValue(subscriptionMap);
- return BoxedUnit.UNIT;
- }
- });
- }
-
- @Override
- public Future<BoxedUnit> advanceCommitPosition(String subscriberId, DLSN newPosition) {
- return getSubscriber(subscriberId).advanceCommitPosition(newPosition);
- }
-
- @Override
- public Future<Boolean> deleteSubscriber(String subscriberId) {
- subscribers.remove(subscriberId);
- String path = getSubscriberZKPath(subscriberId);
- return Utils.zkDeleteIfNotExist(zkc, path, new ZkVersion(-1));
- }
-
- @Override
- public void close() throws IOException {
- // no-op
- for (SubscriptionStateStore store : subscribers.values()) {
- store.close();
- }
- }
-
-}
[35/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogHandler.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogHandler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogHandler.java
deleted file mode 100644
index 0cf8ed5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogHandler.java
+++ /dev/null
@@ -1,715 +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.Stopwatch;
-import com.twitter.distributedlog.callback.LogSegmentNamesListener;
-import com.twitter.distributedlog.exceptions.LogEmptyException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.io.AsyncAbortable;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.logsegment.PerStreamLogSegmentCache;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * The base class about log handler on managing log segments.
- *
- * <h3>Metrics</h3>
- * The log handler is a base class on managing log segments. so all the metrics
- * here are related to log segments retrieval and exposed under `logsegments`.
- * These metrics are all OpStats, in the format of <code>`scope`/logsegments/`op`</code>.
- * <p>
- * Those operations are:
- * <ul>
- * <li>get_inprogress_segment: time between the inprogress log segment created and
- * the handler read it.
- * <li>get_completed_segment: time between a log segment is turned to completed and
- * the handler read it.
- * <li>negative_get_inprogress_segment: record the negative values for `get_inprogress_segment`.
- * <li>negative_get_completed_segment: record the negative values for `get_completed_segment`.
- * <li>recover_last_entry: recovering last entry from a log segment
- * <li>recover_scanned_entries: the number of entries that are scanned during recovering.
- * </ul>
- * @see BKLogWriteHandler
- * @see BKLogReadHandler
- */
-public abstract class BKLogHandler implements AsyncCloseable, AsyncAbortable {
- static final Logger LOG = LoggerFactory.getLogger(BKLogHandler.class);
-
- protected final LogMetadata logMetadata;
- protected final DistributedLogConfiguration conf;
- protected final LogStreamMetadataStore streamMetadataStore;
- protected final LogSegmentMetadataStore metadataStore;
- protected final LogSegmentMetadataCache metadataCache;
- protected final LogSegmentEntryStore entryStore;
- protected final int firstNumEntriesPerReadLastRecordScan;
- protected final int maxNumEntriesPerReadLastRecordScan;
- protected volatile long lastLedgerRollingTimeMillis = -1;
- protected final OrderedScheduler scheduler;
- protected final StatsLogger statsLogger;
- protected final AlertStatsLogger alertStatsLogger;
- protected volatile boolean reportGetSegmentStats = false;
- private final String lockClientId;
- protected final AtomicReference<IOException> metadataException = new AtomicReference<IOException>(null);
-
- // Maintain the list of log segments per stream
- protected final PerStreamLogSegmentCache logSegmentCache;
-
- // trace
- protected final long metadataLatencyWarnThresholdMillis;
-
- // Stats
- private final OpStatsLogger getInprogressSegmentStat;
- private final OpStatsLogger getCompletedSegmentStat;
- private final OpStatsLogger negativeGetInprogressSegmentStat;
- private final OpStatsLogger negativeGetCompletedSegmentStat;
- private final OpStatsLogger recoverLastEntryStats;
- private final OpStatsLogger recoverScannedEntriesStats;
-
- /**
- * Construct a Bookkeeper journal manager.
- */
- BKLogHandler(LogMetadata metadata,
- DistributedLogConfiguration conf,
- LogStreamMetadataStore streamMetadataStore,
- LogSegmentMetadataCache metadataCache,
- LogSegmentEntryStore entryStore,
- OrderedScheduler scheduler,
- StatsLogger statsLogger,
- AlertStatsLogger alertStatsLogger,
- String lockClientId) {
- this.logMetadata = metadata;
- this.conf = conf;
- this.scheduler = scheduler;
- this.statsLogger = statsLogger;
- this.alertStatsLogger = alertStatsLogger;
- this.logSegmentCache = new PerStreamLogSegmentCache(
- metadata.getLogName(),
- conf.isLogSegmentSequenceNumberValidationEnabled());
- firstNumEntriesPerReadLastRecordScan = conf.getFirstNumEntriesPerReadLastRecordScan();
- maxNumEntriesPerReadLastRecordScan = conf.getMaxNumEntriesPerReadLastRecordScan();
- this.streamMetadataStore = streamMetadataStore;
- this.metadataStore = streamMetadataStore.getLogSegmentMetadataStore();
- this.metadataCache = metadataCache;
- this.entryStore = entryStore;
- this.lockClientId = lockClientId;
-
- // Traces
- this.metadataLatencyWarnThresholdMillis = conf.getMetadataLatencyWarnThresholdMillis();
-
- // Stats
- StatsLogger segmentsLogger = statsLogger.scope("logsegments");
- getInprogressSegmentStat = segmentsLogger.getOpStatsLogger("get_inprogress_segment");
- getCompletedSegmentStat = segmentsLogger.getOpStatsLogger("get_completed_segment");
- negativeGetInprogressSegmentStat = segmentsLogger.getOpStatsLogger("negative_get_inprogress_segment");
- negativeGetCompletedSegmentStat = segmentsLogger.getOpStatsLogger("negative_get_completed_segment");
- recoverLastEntryStats = segmentsLogger.getOpStatsLogger("recover_last_entry");
- recoverScannedEntriesStats = segmentsLogger.getOpStatsLogger("recover_scanned_entries");
- }
-
- BKLogHandler checkMetadataException() throws IOException {
- if (null != metadataException.get()) {
- throw metadataException.get();
- }
- return this;
- }
-
- public void reportGetSegmentStats(boolean enabled) {
- this.reportGetSegmentStats = enabled;
- }
-
- public String getLockClientId() {
- return lockClientId;
- }
-
- public Future<LogRecordWithDLSN> asyncGetFirstLogRecord() {
- final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
- streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
- .addEventListener(new FutureEventListener<Void>() {
- @Override
- public void onSuccess(Void value) {
- readLogSegmentsFromStore(
- LogSegmentMetadata.COMPARATOR,
- LogSegmentFilter.DEFAULT_FILTER,
- null
- ).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
-
- @Override
- public void onSuccess(Versioned<List<LogSegmentMetadata>> ledgerList) {
- if (ledgerList.getValue().isEmpty()) {
- promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
- return;
- }
- Future<LogRecordWithDLSN> firstRecord = null;
- for (LogSegmentMetadata ledger : ledgerList.getValue()) {
- if (!ledger.isTruncated() && (ledger.getRecordCount() > 0 || ledger.isInProgress())) {
- firstRecord = asyncReadFirstUserRecord(ledger, DLSN.InitialDLSN);
- break;
- }
- }
- if (null != firstRecord) {
- promise.become(firstRecord);
- } else {
- promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- promise.setException(cause);
- }
- });
- }
-
- @Override
- public void onFailure(Throwable cause) {
- promise.setException(cause);
- }
- });
- return promise;
- }
-
- public Future<LogRecordWithDLSN> getLastLogRecordAsync(final boolean recover, final boolean includeEndOfStream) {
- final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
- streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
- .addEventListener(new FutureEventListener<Void>() {
- @Override
- public void onSuccess(Void value) {
- readLogSegmentsFromStore(
- LogSegmentMetadata.DESC_COMPARATOR,
- LogSegmentFilter.DEFAULT_FILTER,
- null
- ).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
-
- @Override
- public void onSuccess(Versioned<List<LogSegmentMetadata>> ledgerList) {
- if (ledgerList.getValue().isEmpty()) {
- promise.setException(
- new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
- return;
- }
- asyncGetLastLogRecord(
- ledgerList.getValue().iterator(),
- promise,
- recover,
- false,
- includeEndOfStream);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- promise.setException(cause);
- }
- });
- }
-
- @Override
- public void onFailure(Throwable cause) {
- promise.setException(cause);
- }
- });
- return promise;
- }
-
- private void asyncGetLastLogRecord(final Iterator<LogSegmentMetadata> ledgerIter,
- final Promise<LogRecordWithDLSN> promise,
- final boolean fence,
- final boolean includeControlRecord,
- final boolean includeEndOfStream) {
- if (ledgerIter.hasNext()) {
- LogSegmentMetadata metadata = ledgerIter.next();
- asyncReadLastRecord(metadata, fence, includeControlRecord, includeEndOfStream).addEventListener(
- new FutureEventListener<LogRecordWithDLSN>() {
- @Override
- public void onSuccess(LogRecordWithDLSN record) {
- if (null == record) {
- asyncGetLastLogRecord(ledgerIter, promise, fence, includeControlRecord, includeEndOfStream);
- } else {
- promise.setValue(record);
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- promise.setException(cause);
- }
- }
- );
- } else {
- promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
- }
- }
-
- private Future<LogRecordWithDLSN> asyncReadFirstUserRecord(LogSegmentMetadata ledger, DLSN beginDLSN) {
- return ReadUtils.asyncReadFirstUserRecord(
- getFullyQualifiedName(),
- ledger,
- firstNumEntriesPerReadLastRecordScan,
- maxNumEntriesPerReadLastRecordScan,
- new AtomicInteger(0),
- scheduler,
- entryStore,
- beginDLSN
- );
- }
-
- /**
- * This is a helper method to compactly return the record count between two records, the first denoted by
- * beginDLSN and the second denoted by endPosition. Its up to the caller to ensure that endPosition refers to
- * position in the same ledger as beginDLSN.
- */
- private Future<Long> asyncGetLogRecordCount(LogSegmentMetadata ledger, final DLSN beginDLSN, final long endPosition) {
- return asyncReadFirstUserRecord(ledger, beginDLSN).map(new Function<LogRecordWithDLSN, Long>() {
- public Long apply(final LogRecordWithDLSN beginRecord) {
- long recordCount = 0;
- if (null != beginRecord) {
- recordCount = endPosition + 1 - beginRecord.getLastPositionWithinLogSegment();
- }
- return recordCount;
- }
- });
- }
-
- /**
- * Ledger metadata tells us how many records are in each completed segment, but for the first and last segments
- * we may have to crack open the entry and count. For the first entry, we need to do so because beginDLSN may be
- * an interior entry. For the last entry, if it is inprogress, we need to recover it and find the last user
- * entry.
- */
- private Future<Long> asyncGetLogRecordCount(final LogSegmentMetadata ledger, final DLSN beginDLSN) {
- if (ledger.isInProgress() && ledger.isDLSNinThisSegment(beginDLSN)) {
- return asyncReadLastUserRecord(ledger).flatMap(new Function<LogRecordWithDLSN, Future<Long>>() {
- public Future<Long> apply(final LogRecordWithDLSN endRecord) {
- if (null != endRecord) {
- return asyncGetLogRecordCount(ledger, beginDLSN, endRecord.getLastPositionWithinLogSegment() /* end position */);
- } else {
- return Future.value((long) 0);
- }
- }
- });
- } else if (ledger.isInProgress()) {
- return asyncReadLastUserRecord(ledger).map(new Function<LogRecordWithDLSN, Long>() {
- public Long apply(final LogRecordWithDLSN endRecord) {
- if (null != endRecord) {
- return (long) endRecord.getLastPositionWithinLogSegment();
- } else {
- return (long) 0;
- }
- }
- });
- } else if (ledger.isDLSNinThisSegment(beginDLSN)) {
- return asyncGetLogRecordCount(ledger, beginDLSN, ledger.getRecordCount() /* end position */);
- } else {
- return Future.value((long) ledger.getRecordCount());
- }
- }
-
- /**
- * Get a count of records between beginDLSN and the end of the stream.
- *
- * @param beginDLSN dlsn marking the start of the range
- * @return the count of records present in the range
- */
- public Future<Long> asyncGetLogRecordCount(final DLSN beginDLSN) {
- return streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
- .flatMap(new Function<Void, Future<Long>>() {
- public Future<Long> apply(Void done) {
-
- return readLogSegmentsFromStore(
- LogSegmentMetadata.COMPARATOR,
- LogSegmentFilter.DEFAULT_FILTER,
- null
- ).flatMap(new Function<Versioned<List<LogSegmentMetadata>>, Future<Long>>() {
- public Future<Long> apply(Versioned<List<LogSegmentMetadata>> ledgerList) {
-
- List<Future<Long>> futureCounts = new ArrayList<Future<Long>>(ledgerList.getValue().size());
- for (LogSegmentMetadata ledger : ledgerList.getValue()) {
- if (ledger.getLogSegmentSequenceNumber() >= beginDLSN.getLogSegmentSequenceNo()) {
- futureCounts.add(asyncGetLogRecordCount(ledger, beginDLSN));
- }
- }
- return Future.collect(futureCounts).map(new Function<List<Long>, Long>() {
- public Long apply(List<Long> counts) {
- return sum(counts);
- }
- });
- }
- });
- }
- });
- }
-
- private Long sum(List<Long> values) {
- long sum = 0;
- for (Long value : values) {
- sum += value;
- }
- return sum;
- }
-
- @Override
- public Future<Void> asyncAbort() {
- return asyncClose();
- }
-
- public Future<LogRecordWithDLSN> asyncReadLastUserRecord(final LogSegmentMetadata l) {
- return asyncReadLastRecord(l, false, false, false);
- }
-
- public Future<LogRecordWithDLSN> asyncReadLastRecord(final LogSegmentMetadata l,
- final boolean fence,
- final boolean includeControl,
- final boolean includeEndOfStream) {
- final AtomicInteger numRecordsScanned = new AtomicInteger(0);
- final Stopwatch stopwatch = Stopwatch.createStarted();
- return ReadUtils.asyncReadLastRecord(
- getFullyQualifiedName(),
- l,
- fence,
- includeControl,
- includeEndOfStream,
- firstNumEntriesPerReadLastRecordScan,
- maxNumEntriesPerReadLastRecordScan,
- numRecordsScanned,
- scheduler,
- entryStore
- ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
- @Override
- public void onSuccess(LogRecordWithDLSN value) {
- recoverLastEntryStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- recoverScannedEntriesStats.registerSuccessfulEvent(numRecordsScanned.get());
- }
-
- @Override
- public void onFailure(Throwable cause) {
- recoverLastEntryStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- }
- });
- }
-
- protected void setLastLedgerRollingTimeMillis(long rollingTimeMillis) {
- if (lastLedgerRollingTimeMillis < rollingTimeMillis) {
- lastLedgerRollingTimeMillis = rollingTimeMillis;
- }
- }
-
- public String getFullyQualifiedName() {
- return logMetadata.getFullyQualifiedName();
- }
-
- // Log Segments Related Functions
- //
- // ***Note***
- // Get log segment list should go through #getCachedLogSegments as we need to assign start sequence id
- // for inprogress log segment so the reader could generate the right sequence id.
- //
- // ***PerStreamCache vs LogSegmentMetadataCache **
- // The per stream cache maintains the list of segments per stream, while the metadata cache
- // maintains log segments. The metadata cache is just to reduce the access to zookeeper, it is
- // okay that some of the log segments are not in the cache; however the per stream cache can not
- // have any gaps between log segment sequence numbers which it has to be accurate.
-
- /**
- * Get the cached log segments.
- *
- * @param comparator the comparator to sort the returned log segments.
- * @return list of sorted log segments
- * @throws UnexpectedException if unexpected condition detected.
- */
- protected List<LogSegmentMetadata> getCachedLogSegments(Comparator<LogSegmentMetadata> comparator)
- throws UnexpectedException {
- try {
- return logSegmentCache.getLogSegments(comparator);
- } catch (UnexpectedException ue) {
- // the log segments cache went wrong
- LOG.error("Unexpected exception on getting log segments from the cache for stream {}",
- getFullyQualifiedName(), ue);
- metadataException.compareAndSet(null, ue);
- throw ue;
- }
- }
-
- /**
- * Add the segment <i>metadata</i> for <i>name</i> in the cache.
- *
- * @param name
- * segment znode name.
- * @param metadata
- * segment metadata.
- */
- protected void addLogSegmentToCache(String name, LogSegmentMetadata metadata) {
- metadataCache.put(metadata.getZkPath(), metadata);
- logSegmentCache.add(name, metadata);
- // update the last ledger rolling time
- if (!metadata.isInProgress() && (lastLedgerRollingTimeMillis < metadata.getCompletionTime())) {
- lastLedgerRollingTimeMillis = metadata.getCompletionTime();
- }
-
- if (reportGetSegmentStats) {
- // update stats
- long ts = System.currentTimeMillis();
- if (metadata.isInProgress()) {
- // as we used timestamp as start tx id we could take it as start time
- // NOTE: it is a hack here.
- long elapsedMillis = ts - metadata.getFirstTxId();
- long elapsedMicroSec = TimeUnit.MILLISECONDS.toMicros(elapsedMillis);
- if (elapsedMicroSec > 0) {
- if (elapsedMillis > metadataLatencyWarnThresholdMillis) {
- LOG.warn("{} received inprogress log segment in {} millis: {}",
- new Object[] { getFullyQualifiedName(), elapsedMillis, metadata });
- }
- getInprogressSegmentStat.registerSuccessfulEvent(elapsedMicroSec);
- } else {
- negativeGetInprogressSegmentStat.registerSuccessfulEvent(-elapsedMicroSec);
- }
- } else {
- long elapsedMillis = ts - metadata.getCompletionTime();
- long elapsedMicroSec = TimeUnit.MILLISECONDS.toMicros(elapsedMillis);
- if (elapsedMicroSec > 0) {
- if (elapsedMillis > metadataLatencyWarnThresholdMillis) {
- LOG.warn("{} received completed log segment in {} millis : {}",
- new Object[] { getFullyQualifiedName(), elapsedMillis, metadata });
- }
- getCompletedSegmentStat.registerSuccessfulEvent(elapsedMicroSec);
- } else {
- negativeGetCompletedSegmentStat.registerSuccessfulEvent(-elapsedMicroSec);
- }
- }
- }
- }
-
- /**
- * Read log segment <i>name</i> from the cache.
- *
- * @param name name of the log segment
- * @return log segment metadata
- */
- protected LogSegmentMetadata readLogSegmentFromCache(String name) {
- return logSegmentCache.get(name);
- }
-
- /**
- * Remove the log segment <i>name</i> from the cache.
- *
- * @param name name of the log segment.
- * @return log segment metadata
- */
- protected LogSegmentMetadata removeLogSegmentFromCache(String name) {
- metadataCache.invalidate(name);
- return logSegmentCache.remove(name);
- }
-
- /**
- * Update the log segment cache with updated mapping
- *
- * @param logSegmentsRemoved log segments removed
- * @param logSegmentsAdded log segments added
- */
- protected void updateLogSegmentCache(Set<String> logSegmentsRemoved,
- Map<String, LogSegmentMetadata> logSegmentsAdded) {
- for (String segmentName : logSegmentsRemoved) {
- metadataCache.invalidate(segmentName);
- }
- for (Map.Entry<String, LogSegmentMetadata> entry : logSegmentsAdded.entrySet()) {
- metadataCache.put(entry.getKey(), entry.getValue());
- }
- logSegmentCache.update(logSegmentsRemoved, logSegmentsAdded);
- }
-
- /**
- * Read the log segments from the store and register a listener
- * @param comparator
- * @param segmentFilter
- * @param logSegmentNamesListener
- * @return future represents the result of log segments
- */
- public Future<Versioned<List<LogSegmentMetadata>>> readLogSegmentsFromStore(
- final Comparator<LogSegmentMetadata> comparator,
- final LogSegmentFilter segmentFilter,
- final LogSegmentNamesListener logSegmentNamesListener) {
- final Promise<Versioned<List<LogSegmentMetadata>>> readResult =
- new Promise<Versioned<List<LogSegmentMetadata>>>();
- metadataStore.getLogSegmentNames(logMetadata.getLogSegmentsPath(), logSegmentNamesListener)
- .addEventListener(new FutureEventListener<Versioned<List<String>>>() {
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(readResult, cause);
- }
-
- @Override
- public void onSuccess(Versioned<List<String>> logSegmentNames) {
- readLogSegmentsFromStore(logSegmentNames, comparator, segmentFilter, readResult);
- }
- });
- return readResult;
- }
-
- protected void readLogSegmentsFromStore(final Versioned<List<String>> logSegmentNames,
- final Comparator<LogSegmentMetadata> comparator,
- final LogSegmentFilter segmentFilter,
- final Promise<Versioned<List<LogSegmentMetadata>>> readResult) {
- Set<String> segmentsReceived = new HashSet<String>();
- segmentsReceived.addAll(segmentFilter.filter(logSegmentNames.getValue()));
- Set<String> segmentsAdded;
- final Set<String> removedSegments = Collections.synchronizedSet(new HashSet<String>());
- final Map<String, LogSegmentMetadata> addedSegments =
- Collections.synchronizedMap(new HashMap<String, LogSegmentMetadata>());
- Pair<Set<String>, Set<String>> segmentChanges = logSegmentCache.diff(segmentsReceived);
- segmentsAdded = segmentChanges.getLeft();
- removedSegments.addAll(segmentChanges.getRight());
-
- if (segmentsAdded.isEmpty()) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("No segments added for {}.", getFullyQualifiedName());
- }
-
- // update the cache before #getCachedLogSegments to return
- updateLogSegmentCache(removedSegments, addedSegments);
-
- List<LogSegmentMetadata> segmentList;
- try {
- segmentList = getCachedLogSegments(comparator);
- } catch (UnexpectedException e) {
- FutureUtils.setException(readResult, e);
- return;
- }
-
- FutureUtils.setValue(readResult,
- new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNames.getVersion()));
- return;
- }
-
- final AtomicInteger numChildren = new AtomicInteger(segmentsAdded.size());
- final AtomicInteger numFailures = new AtomicInteger(0);
- for (final String segment: segmentsAdded) {
- String logSegmentPath = logMetadata.getLogSegmentPath(segment);
- LogSegmentMetadata cachedSegment = metadataCache.get(logSegmentPath);
- if (null != cachedSegment) {
- addedSegments.put(segment, cachedSegment);
- completeReadLogSegmentsFromStore(
- removedSegments,
- addedSegments,
- comparator,
- readResult,
- logSegmentNames.getVersion(),
- numChildren,
- numFailures);
- continue;
- }
- metadataStore.getLogSegment(logSegmentPath)
- .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
-
- @Override
- public void onSuccess(LogSegmentMetadata result) {
- addedSegments.put(segment, result);
- complete();
- }
-
- @Override
- public void onFailure(Throwable cause) {
- // LogSegmentNotFoundException exception is possible in two cases
- // 1. A log segment was deleted by truncation between the call to getChildren and read
- // attempt on the znode corresponding to the segment
- // 2. In progress segment has been completed => inprogress ZNode does not exist
- if (cause instanceof LogSegmentNotFoundException) {
- removedSegments.add(segment);
- complete();
- } else {
- // fail fast
- if (1 == numFailures.incrementAndGet()) {
- FutureUtils.setException(readResult, cause);
- return;
- }
- }
- }
-
- private void complete() {
- completeReadLogSegmentsFromStore(
- removedSegments,
- addedSegments,
- comparator,
- readResult,
- logSegmentNames.getVersion(),
- numChildren,
- numFailures);
- }
- });
- }
- }
-
- private void completeReadLogSegmentsFromStore(final Set<String> removedSegments,
- final Map<String, LogSegmentMetadata> addedSegments,
- final Comparator<LogSegmentMetadata> comparator,
- final Promise<Versioned<List<LogSegmentMetadata>>> readResult,
- final Version logSegmentNamesVersion,
- final AtomicInteger numChildren,
- final AtomicInteger numFailures) {
- if (0 != numChildren.decrementAndGet()) {
- return;
- }
- if (numFailures.get() > 0) {
- return;
- }
- // update the cache only when fetch completed and before #getCachedLogSegments
- updateLogSegmentCache(removedSegments, addedSegments);
- List<LogSegmentMetadata> segmentList;
- try {
- segmentList = getCachedLogSegments(comparator);
- } catch (UnexpectedException e) {
- FutureUtils.setException(readResult, e);
- return;
- }
- FutureUtils.setValue(readResult,
- new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNamesVersion));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogReadHandler.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogReadHandler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogReadHandler.java
deleted file mode 100644
index 8aa00e7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogReadHandler.java
+++ /dev/null
@@ -1,431 +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 java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.callback.LogSegmentListener;
-import com.twitter.distributedlog.callback.LogSegmentNamesListener;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.metadata.LogMetadataForReader;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.ExceptionalFunction;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import com.twitter.util.Try;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.SafeRunnable;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import javax.annotation.Nullable;
-
-/**
- * Log Handler for Readers.
- * <h3>Metrics</h3>
- *
- * <h4>ReadAhead Worker</h4>
- * Most of readahead stats are exposed under scope `readahead_worker`. Only readahead exceptions are exposed
- * in parent scope via <code>readAheadExceptionsLogger</code>.
- * <ul>
- * <li> `readahead_worker`/wait: counter. number of waits that readahead worker is waiting. If this keeps increasing,
- * it usually means readahead keep getting full because of reader slows down reading.
- * <li> `readahead_worker`/repositions: counter. number of repositions that readhead worker encounters. reposition
- * means that a readahead worker finds that it isn't advancing to a new log segment and force re-positioning.
- * <li> `readahead_worker`/entry_piggy_back_hits: counter. it increases when the last add confirmed being advanced
- * because of the piggy-back lac.
- * <li> `readahead_worker`/entry_piggy_back_misses: counter. it increases when the last add confirmed isn't advanced
- * by a read entry because it doesn't piggy back a newer lac.
- * <li> `readahead_worker`/read_entries: opstats. stats on number of entries read per readahead read batch.
- * <li> `readahead_worker`/read_lac_counter: counter. stats on the number of readLastConfirmed operations
- * <li> `readahead_worker`/read_lac_and_entry_counter: counter. stats on the number of readLastConfirmedAndEntry
- * operations.
- * <li> `readahead_worker`/cache_full: counter. it increases each time readahead worker finds cache become full.
- * If it keeps increasing, that means reader slows down reading.
- * <li> `readahead_worker`/resume: opstats. stats on readahead worker resuming reading from wait state.
- * <li> `readahead_worker`/read_lac_lag: opstats. stats on the number of entries diff between the lac reader knew
- * last time and the lac that it received. if `lag` between two subsequent lacs is high, that might means delay
- * might be high. because reader is only allowed to read entries after lac is advanced.
- * <li> `readahead_worker`/long_poll_interruption: opstats. stats on the number of interruptions happened to long
- * poll. the interruptions are usually because of receiving zookeeper notifications.
- * <li> `readahead_worker`/notification_execution: opstats. stats on executions over the notifications received from
- * zookeeper.
- * <li> `readahead_worker`/metadata_reinitialization: opstats. stats on metadata reinitialization after receiving
- * notifcation from log segments updates.
- * <li> `readahead_worker`/idle_reader_warn: counter. it increases each time the readahead worker detects itself
- * becoming idle.
- * </ul>
- * <h4>Read Lock</h4>
- * All read lock related stats are exposed under scope `read_lock`.
- * for detail stats.
- */
-class BKLogReadHandler extends BKLogHandler implements LogSegmentNamesListener {
- static final Logger LOG = LoggerFactory.getLogger(BKLogReadHandler.class);
-
- protected final LogMetadataForReader logMetadataForReader;
-
- protected final DynamicDistributedLogConfiguration dynConf;
-
- private final Optional<String> subscriberId;
- private DistributedLock readLock;
- private Future<Void> lockAcquireFuture;
-
- // notify the state change about the read handler
- protected final AsyncNotification readerStateNotification;
-
- // log segments listener
- protected boolean logSegmentsNotificationDisabled = false;
- protected final CopyOnWriteArraySet<LogSegmentListener> listeners =
- new CopyOnWriteArraySet<LogSegmentListener>();
- protected Versioned<List<LogSegmentMetadata>> lastNotifiedLogSegments =
- new Versioned<List<LogSegmentMetadata>>(null, Version.NEW);
-
- // stats
- private final StatsLogger perLogStatsLogger;
-
- /**
- * Construct a Bookkeeper journal manager.
- */
- BKLogReadHandler(LogMetadataForReader logMetadata,
- Optional<String> subscriberId,
- DistributedLogConfiguration conf,
- DynamicDistributedLogConfiguration dynConf,
- LogStreamMetadataStore streamMetadataStore,
- LogSegmentMetadataCache metadataCache,
- LogSegmentEntryStore entryStore,
- OrderedScheduler scheduler,
- AlertStatsLogger alertStatsLogger,
- StatsLogger statsLogger,
- StatsLogger perLogStatsLogger,
- String clientId,
- AsyncNotification readerStateNotification,
- boolean isHandleForReading) {
- super(logMetadata,
- conf,
- streamMetadataStore,
- metadataCache,
- entryStore,
- scheduler,
- statsLogger,
- alertStatsLogger,
- clientId);
- this.logMetadataForReader = logMetadata;
- this.dynConf = dynConf;
- this.perLogStatsLogger =
- isHandleForReading ? perLogStatsLogger : NullStatsLogger.INSTANCE;
- this.readerStateNotification = readerStateNotification;
- this.subscriberId = subscriberId;
- }
-
- @VisibleForTesting
- String getReadLockPath() {
- return logMetadataForReader.getReadLockPath(subscriberId);
- }
-
- <T> void satisfyPromiseAsync(final Promise<T> promise, final Try<T> result) {
- scheduler.submit(new SafeRunnable() {
- @Override
- public void safeRun() {
- promise.update(result);
- }
- });
- }
-
- Future<Void> checkLogStreamExists() {
- return streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName());
- }
-
- /**
- * Elective stream lock--readers are not required to acquire the lock before using the stream.
- */
- synchronized Future<Void> lockStream() {
- if (null == lockAcquireFuture) {
- lockAcquireFuture = streamMetadataStore.createReadLock(logMetadataForReader, subscriberId)
- .flatMap(new ExceptionalFunction<DistributedLock, Future<Void>>() {
- @Override
- public Future<Void> applyE(DistributedLock lock) throws Throwable {
- BKLogReadHandler.this.readLock = lock;
- LOG.info("acquiring readlock {} at {}", getLockClientId(), getReadLockPath());
- return acquireLockOnExecutorThread(lock);
- }
- });
- }
- return lockAcquireFuture;
- }
-
- /**
- * Begin asynchronous lock acquire, but ensure that the returned future is satisfied on an
- * executor service thread.
- */
- Future<Void> acquireLockOnExecutorThread(DistributedLock lock) throws LockingException {
- final Future<? extends DistributedLock> acquireFuture = lock.asyncAcquire();
-
- // The future we return must be satisfied on an executor service thread. If we simply
- // return the future returned by asyncAcquire, user callbacks may end up running in
- // the lock state executor thread, which will cause deadlocks and introduce latency
- // etc.
- final Promise<Void> threadAcquirePromise = new Promise<Void>();
- threadAcquirePromise.setInterruptHandler(new Function<Throwable, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Throwable t) {
- FutureUtils.cancel(acquireFuture);
- return null;
- }
- });
- acquireFuture.addEventListener(new FutureEventListener<DistributedLock>() {
- @Override
- public void onSuccess(DistributedLock lock) {
- LOG.info("acquired readlock {} at {}", getLockClientId(), getReadLockPath());
- satisfyPromiseAsync(threadAcquirePromise, new Return<Void>(null));
- }
-
- @Override
- public void onFailure(Throwable cause) {
- LOG.info("failed to acquire readlock {} at {}",
- new Object[]{ getLockClientId(), getReadLockPath(), cause });
- satisfyPromiseAsync(threadAcquirePromise, new Throw<Void>(cause));
- }
- });
- return threadAcquirePromise;
- }
-
- /**
- * Check ownership of elective stream lock.
- */
- void checkReadLock() throws DLIllegalStateException, LockingException {
- synchronized (this) {
- if ((null == lockAcquireFuture) ||
- (!lockAcquireFuture.isDefined())) {
- throw new DLIllegalStateException("Attempt to check for lock before it has been acquired successfully");
- }
- }
-
- readLock.checkOwnership();
- }
-
- public Future<Void> asyncClose() {
- DistributedLock lockToClose;
- synchronized (this) {
- if (null != lockAcquireFuture && !lockAcquireFuture.isDefined()) {
- FutureUtils.cancel(lockAcquireFuture);
- }
- lockToClose = readLock;
- }
- return Utils.closeSequence(scheduler, lockToClose)
- .flatMap(new AbstractFunction1<Void, Future<Void>>() {
- @Override
- public Future<Void> apply(Void result) {
- // unregister the log segment listener
- metadataStore.unregisterLogSegmentListener(logMetadata.getLogSegmentsPath(), BKLogReadHandler.this);
- return Future.Void();
- }
- });
- }
-
- @Override
- public Future<Void> asyncAbort() {
- return asyncClose();
- }
-
- /**
- * Start fetch the log segments and register the {@link LogSegmentNamesListener}.
- * The future is satisfied only on a successful fetch or encountered a fatal failure.
- *
- * @return future represents the fetch result
- */
- Future<Versioned<List<LogSegmentMetadata>>> asyncStartFetchLogSegments() {
- Promise<Versioned<List<LogSegmentMetadata>>> promise =
- new Promise<Versioned<List<LogSegmentMetadata>>>();
- asyncStartFetchLogSegments(promise);
- return promise;
- }
-
- void asyncStartFetchLogSegments(final Promise<Versioned<List<LogSegmentMetadata>>> promise) {
- readLogSegmentsFromStore(
- LogSegmentMetadata.COMPARATOR,
- LogSegmentFilter.DEFAULT_FILTER,
- this).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
- @Override
- public void onFailure(Throwable cause) {
- if (cause instanceof LogNotFoundException ||
- cause instanceof LogSegmentNotFoundException ||
- cause instanceof UnexpectedException) {
- // indicate some inconsistent behavior, abort
- metadataException.compareAndSet(null, (IOException) cause);
- // notify the reader that read handler is in error state
- notifyReaderOnError(cause);
- FutureUtils.setException(promise, cause);
- return;
- }
- scheduler.schedule(new Runnable() {
- @Override
- public void run() {
- asyncStartFetchLogSegments(promise);
- }
- }, conf.getZKRetryBackoffMaxMillis(), TimeUnit.MILLISECONDS);
- }
-
- @Override
- public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
- // no-op
- FutureUtils.setValue(promise, segments);
- }
- });
- }
-
- @VisibleForTesting
- void disableReadAheadLogSegmentsNotification() {
- logSegmentsNotificationDisabled = true;
- }
-
- @Override
- public void onSegmentsUpdated(final Versioned<List<String>> segments) {
- synchronized (this) {
- if (lastNotifiedLogSegments.getVersion() != Version.NEW &&
- lastNotifiedLogSegments.getVersion().compare(segments.getVersion()) != Version.Occurred.BEFORE) {
- // the log segments has been read, and it is possibly a retry from last segments update
- return;
- }
- }
-
- Promise<Versioned<List<LogSegmentMetadata>>> readLogSegmentsPromise =
- new Promise<Versioned<List<LogSegmentMetadata>>>();
- readLogSegmentsPromise.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
- @Override
- public void onFailure(Throwable cause) {
- if (cause instanceof LogNotFoundException ||
- cause instanceof LogSegmentNotFoundException ||
- cause instanceof UnexpectedException) {
- // indicate some inconsistent behavior, abort
- metadataException.compareAndSet(null, (IOException) cause);
- // notify the reader that read handler is in error state
- notifyReaderOnError(cause);
- return;
- }
- scheduler.schedule(new Runnable() {
- @Override
- public void run() {
- onSegmentsUpdated(segments);
- }
- }, conf.getZKRetryBackoffMaxMillis(), TimeUnit.MILLISECONDS);
- }
-
- @Override
- public void onSuccess(Versioned<List<LogSegmentMetadata>> logSegments) {
- List<LogSegmentMetadata> segmentsToNotify = null;
- synchronized (BKLogReadHandler.this) {
- Versioned<List<LogSegmentMetadata>> lastLogSegments = lastNotifiedLogSegments;
- if (lastLogSegments.getVersion() == Version.NEW ||
- lastLogSegments.getVersion().compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
- lastNotifiedLogSegments = logSegments;
- segmentsToNotify = logSegments.getValue();
- }
- }
- if (null != segmentsToNotify) {
- notifyUpdatedLogSegments(segmentsToNotify);
- }
- }
- });
- // log segments list is updated, read their metadata
- readLogSegmentsFromStore(
- segments,
- LogSegmentMetadata.COMPARATOR,
- LogSegmentFilter.DEFAULT_FILTER,
- readLogSegmentsPromise);
- }
-
- @Override
- public void onLogStreamDeleted() {
- notifyLogStreamDeleted();
- }
-
- //
- // Listener for log segments
- //
-
- protected void registerListener(@Nullable LogSegmentListener listener) {
- if (null != listener) {
- listeners.add(listener);
- }
- }
-
- protected void unregisterListener(@Nullable LogSegmentListener listener) {
- if (null != listener) {
- listeners.remove(listener);
- }
- }
-
- protected void notifyUpdatedLogSegments(List<LogSegmentMetadata> segments) {
- if (logSegmentsNotificationDisabled) {
- return;
- }
-
- for (LogSegmentListener listener : listeners) {
- List<LogSegmentMetadata> listToReturn =
- new ArrayList<LogSegmentMetadata>(segments);
- Collections.sort(listToReturn, LogSegmentMetadata.COMPARATOR);
- listener.onSegmentsUpdated(listToReturn);
- }
- }
-
- protected void notifyLogStreamDeleted() {
- if (logSegmentsNotificationDisabled) {
- return;
- }
-
- for (LogSegmentListener listener : listeners) {
- listener.onLogStreamDeleted();
- }
- }
-
- // notify the errors
- protected void notifyReaderOnError(Throwable cause) {
- if (null != readerStateNotification) {
- readerStateNotification.notifyOnError(cause);
- }
- }
-}
[19/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLockFactory.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLockFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLockFactory.java
deleted file mode 100644
index bb98e07..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLockFactory.java
+++ /dev/null
@@ -1,133 +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.lock;
-
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import org.apache.bookkeeper.stats.StatsLogger;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Factory to create zookeeper based locks.
- */
-public class ZKSessionLockFactory implements SessionLockFactory {
-
- private final ZooKeeperClient zkc;
- private final String clientId;
- private final OrderedScheduler lockStateExecutor;
- private final long lockOpTimeout;
- private final int lockCreationRetries;
- private final long zkRetryBackoffMs;
-
- // Stats
- private final StatsLogger lockStatsLogger;
-
- public ZKSessionLockFactory(ZooKeeperClient zkc,
- String clientId,
- OrderedScheduler lockStateExecutor,
- int lockCreationRetries,
- long lockOpTimeout,
- long zkRetryBackoffMs,
- StatsLogger statsLogger) {
- this.zkc = zkc;
- this.clientId = clientId;
- this.lockStateExecutor = lockStateExecutor;
- this.lockCreationRetries = lockCreationRetries;
- this.lockOpTimeout = lockOpTimeout;
- this.zkRetryBackoffMs = zkRetryBackoffMs;
-
- this.lockStatsLogger = statsLogger.scope("lock");
- }
-
- @Override
- public Future<SessionLock> createLock(String lockPath,
- DistributedLockContext context) {
- AtomicInteger numRetries = new AtomicInteger(lockCreationRetries);
- final AtomicReference<Throwable> interruptedException = new AtomicReference<Throwable>(null);
- Promise<SessionLock> createPromise =
- new Promise<SessionLock>(new com.twitter.util.Function<Throwable, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Throwable t) {
- interruptedException.set(t);
- return BoxedUnit.UNIT;
- }
- });
- createLock(
- lockPath,
- context,
- interruptedException,
- numRetries,
- createPromise,
- 0L);
- return createPromise;
- }
-
- void createLock(final String lockPath,
- final DistributedLockContext context,
- final AtomicReference<Throwable> interruptedException,
- final AtomicInteger numRetries,
- final Promise<SessionLock> createPromise,
- final long delayMs) {
- lockStateExecutor.schedule(lockPath, new Runnable() {
- @Override
- public void run() {
- if (null != interruptedException.get()) {
- createPromise.updateIfEmpty(new Throw<SessionLock>(interruptedException.get()));
- return;
- }
- try {
- SessionLock lock = new ZKSessionLock(
- zkc,
- lockPath,
- clientId,
- lockStateExecutor,
- lockOpTimeout,
- lockStatsLogger,
- context);
- createPromise.updateIfEmpty(new Return<SessionLock>(lock));
- } catch (DLInterruptedException dlie) {
- // if the creation is interrupted, throw the exception without retrie.
- createPromise.updateIfEmpty(new Throw<SessionLock>(dlie));
- return;
- } catch (IOException e) {
- if (numRetries.getAndDecrement() < 0) {
- createPromise.updateIfEmpty(new Throw<SessionLock>(e));
- return;
- }
- createLock(
- lockPath,
- context,
- interruptedException,
- numRetries,
- createPromise,
- zkRetryBackoffMs);
- }
- }
- }, delayMs, TimeUnit.MILLISECONDS);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/package-info.java
deleted file mode 100644
index 02d905d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Distributed locking mechanism in distributedlog
- */
-package com.twitter.distributedlog.lock;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 81eb5ed..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
+++ /dev/null
@@ -1,113 +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.logsegment;
-
-import com.google.common.annotations.Beta;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.util.List;
-
-/**
- * An interface class to read the enveloped entry (serialized bytes of
- * {@link com.twitter.distributedlog.Entry}) from a log segment
- */
-@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)}
- */
- 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
- */
- LogSegmentMetadata getSegment();
-
- /**
- * Update the log segment each time when the metadata has changed.
- *
- * @param segment new metadata of the log segment.
- */
- void onLogSegmentMetadataUpdated(LogSegmentMetadata segment);
-
- /**
- * Read next <i>numEntries</i> entries from current log segment.
- * <p>
- * <i>numEntries</i> will be best-effort.
- *
- * @param numEntries num entries to read from current log segment
- * @return A promise that when satisified will contain a non-empty list of entries with their content.
- * @throws {@link com.twitter.distributedlog.exceptions.EndOfLogSegmentException} when
- * read entries beyond the end of a <i>closed</i> log segment.
- */
- Future<List<Entry.Reader>> readNext(int numEntries);
-
- /**
- * Return the last add confirmed entry id (LAC).
- *
- * @return the last add confirmed entry id.
- */
- long getLastAddConfirmed();
-
- /**
- * Is the reader reading beyond last add confirmed.
- *
- * @return true if the reader is reading beyond last add confirmed
- */
- boolean isBeyondLastAddConfirmed();
-
- /**
- * Has the log segment reader caught up with the inprogress log segment.
- *
- * @return true only if the log segment is inprogress and it is caught up, otherwise return false.
- */
- boolean hasCaughtUpOnInprogress();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
----------------------------------------------------------------------
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
deleted file mode 100644
index bcf8129..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
+++ /dev/null
@@ -1,73 +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.logsegment;
-
-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
- */
-@Beta
-public interface LogSegmentEntryStore {
-
- /**
- * Delete the actual log segment from the entry store.
- *
- * @param segment log segment metadata
- * @return future represent the delete result
- */
- Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment);
-
- /**
- * Create a new log segment allocator for allocating log segment entry writers.
- *
- * @param metadata the metadata for the log stream
- * @return future represent the log segment allocator
- */
- Allocator<LogSegmentEntryWriter, Object> newLogSegmentAllocator(
- LogMetadataForWriter metadata,
- DynamicDistributedLogConfiguration dynConf) throws IOException;
-
- /**
- * Open the reader for reading data to the log <i>segment</i>.
- *
- * @param segment the log <i>segment</i> to read data from
- * @param startEntryId the start entry id
- * @return future represent the opened reader
- */
- Future<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
- long startEntryId);
-
- /**
- * Open the reader for reading entries from a random access log <i>segment</i>.
- *
- * @param segment the log <i>segment</i> to read entries from
- * @param fence the flag to fence log segment
- * @return future represent the opened random access reader
- */
- Future<LogSegmentRandomAccessEntryReader> openRandomAccessReader(LogSegmentMetadata segment,
- boolean fence);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryWriter.java
deleted file mode 100644
index 8b7d9b2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryWriter.java
+++ /dev/null
@@ -1,72 +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.logsegment;
-
-import com.google.common.annotations.Beta;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.util.Sizable;
-import org.apache.bookkeeper.client.AsyncCallback;
-
-/**
- * An interface class to write the enveloped entry (serialized bytes of
- * {@link Entry} into the log segment.
- *
- * <p>It is typically used by {@link LogSegmentWriter}.
- *
- * @see LogSegmentWriter
- *
- * TODO: The interface is leveraging bookkeeper's callback and status code now
- * Consider making it more generic.
- */
-@Beta
-public interface LogSegmentEntryWriter extends Sizable {
-
- /**
- * Get the log segment id.
- *
- * @return log segment id.
- */
- long getLogSegmentId();
-
- /**
- * Close the entry writer.
- */
- void asyncClose(AsyncCallback.CloseCallback callback, Object ctx);
-
- /**
- * Async add entry to the log segment.
- * <p>The implementation semantic follows
- * {@link org.apache.bookkeeper.client.LedgerHandle#asyncAddEntry(
- * byte[], int, int, AsyncCallback.AddCallback, Object)}
- *
- * @param data
- * data to add
- * @param offset
- * offset in the data
- * @param length
- * length of the data
- * @param callback
- * callback
- * @param ctx
- * ctx
- * @see org.apache.bookkeeper.client.LedgerHandle#asyncAddEntry(
- * byte[], int, int, AsyncCallback.AddCallback, Object)
- */
- void asyncAddEntry(byte[] data, int offset, int length,
- AsyncCallback.AddCallback callback, Object ctx);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentFilter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentFilter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentFilter.java
deleted file mode 100644
index f8bf183..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentFilter.java
+++ /dev/null
@@ -1,42 +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.logsegment;
-
-import java.util.Collection;
-
-/**
- * Filter to filter log segments
- */
-public interface LogSegmentFilter {
-
- public static final LogSegmentFilter DEFAULT_FILTER = new LogSegmentFilter() {
- @Override
- public Collection<String> filter(Collection<String> fullList) {
- return fullList;
- }
- };
-
- /**
- * Filter the log segments from the full log segment list.
- *
- * @param fullList
- * full list of log segment names.
- * @return filtered log segment names
- */
- Collection<String> filter(Collection<String> fullList);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataCache.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataCache.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataCache.java
deleted file mode 100644
index d4ca3ea..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataCache.java
+++ /dev/null
@@ -1,98 +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.logsegment;
-
-import com.google.common.base.Ticker;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.RemovalListener;
-import com.google.common.cache.RemovalNotification;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Cache the log segment metadata
- */
-public class LogSegmentMetadataCache implements RemovalListener<String, LogSegmentMetadata> {
-
- private static final Logger logger = LoggerFactory.getLogger(LogSegmentMetadataCache.class);
-
- private final Cache<String, LogSegmentMetadata> cache;
- private final boolean isCacheEnabled;
-
- public LogSegmentMetadataCache(DistributedLogConfiguration conf,
- Ticker ticker) {
- cache = CacheBuilder.newBuilder()
- .concurrencyLevel(conf.getNumWorkerThreads())
- .initialCapacity(1024)
- .expireAfterAccess(conf.getLogSegmentCacheTTLMs(), TimeUnit.MILLISECONDS)
- .maximumSize(conf.getLogSegmentCacheMaxSize())
- .removalListener(this)
- .ticker(ticker)
- .recordStats()
- .build();
- this.isCacheEnabled = conf.isLogSegmentCacheEnabled();
- logger.info("Log segment cache is enabled = {}", this.isCacheEnabled);
- }
-
- /**
- * Add the log <i>segment</i> of <i>path</i> to the cache.
- *
- * @param path the path of the log segment
- * @param segment log segment metadata
- */
- public void put(String path, LogSegmentMetadata segment) {
- if (isCacheEnabled) {
- cache.put(path, segment);
- }
- }
-
- /**
- * Invalid the cache entry associated with <i>path</i>.
- *
- * @param path the path of the log segment
- */
- public void invalidate(String path) {
- if (isCacheEnabled) {
- cache.invalidate(path);
- }
- }
-
- /**
- * Retrieve the log segment of <i>path</i> from the cache.
- *
- * @param path the path of the log segment.
- * @return log segment metadata if exists, otherwise null.
- */
- public LogSegmentMetadata get(String path) {
- return cache.getIfPresent(path);
- }
-
- @Override
- public void onRemoval(RemovalNotification<String, LogSegmentMetadata> notification) {
- if (notification.wasEvicted()) {
- if (logger.isDebugEnabled()) {
- logger.debug("Log segment of {} was evicted.", notification.getKey());
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataStore.java
deleted file mode 100644
index dda76e5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataStore.java
+++ /dev/null
@@ -1,163 +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.logsegment;
-
-import com.google.common.annotations.Beta;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.callback.LogSegmentNamesListener;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-
-import java.io.Closeable;
-import java.util.List;
-
-/**
- * Interface for log segment metadata store. All operations that modify log segments should
- * be executed under a {@link Transaction}.
- */
-@Beta
-public interface LogSegmentMetadataStore extends Closeable {
-
- /**
- * Start the transaction on changing log segment metadata store.
- *
- * @return transaction of the log segment metadata store.
- */
- Transaction<Object> transaction();
-
- // The reason to keep storing log segment sequence number & log record transaction id
- // in this log segment metadata store interface is to share the transaction that used
- // to start/complete log segment. It is a bit hard to separate them out right now.
-
- /**
- * Store the maximum log segment sequence number on <code>path</code>.
- *
- * @param txn
- * transaction to execute for storing log segment sequence number.
- * @param logMetadata
- * metadata of the log stream
- * @param sequenceNumber
- * log segment sequence number to store
- * @param listener
- * listener on the result to this operation
- */
- void storeMaxLogSegmentSequenceNumber(Transaction<Object> txn,
- LogMetadata logMetadata,
- Versioned<Long> sequenceNumber,
- OpListener<Version> listener);
-
- /**
- * Store the maximum transaction id for <code>path</code>
- *
- * @param txn
- * transaction to execute for storing transaction id
- * @param logMetadata
- * metadata of the log stream
- * @param transactionId
- * transaction id to store
- * @param listener
- * listener on the result to this operation
- */
- void storeMaxTxnId(Transaction<Object> txn,
- LogMetadataForWriter logMetadata,
- Versioned<Long> transactionId,
- OpListener<Version> listener);
-
- /**
- * Create a log segment <code>segment</code> under transaction <code>txn</code>.
- *
- * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
- * immediately. the operation should be executed via {@link Transaction#execute()}
- *
- * @param txn
- * transaction to execute for this operation
- * @param segment
- * segment to create
- * @param opListener
- * the listener on the operation result
- */
- void createLogSegment(Transaction<Object> txn,
- LogSegmentMetadata segment,
- OpListener<Void> opListener);
-
- /**
- * Delete a log segment <code>segment</code> under transaction <code>txn</code>.
- *
- * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
- * immediately. the operation should be executed via {@link Transaction#execute()}
- *
- * @param txn
- * transaction to execute for this operation
- * @param segment
- * segment to delete
- */
- void deleteLogSegment(Transaction<Object> txn,
- LogSegmentMetadata segment,
- OpListener<Void> opListener);
-
- /**
- * Update a log segment <code>segment</code> under transaction <code>txn</code>.
- *
- * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
- * immediately. the operation should be executed via {@link Transaction#execute()}
- *
- * @param txn
- * transaction to execute for this operation
- * @param segment
- * segment to update
- */
- void updateLogSegment(Transaction<Object> txn, LogSegmentMetadata segment);
-
- /**
- * Retrieve the log segment associated <code>path</code>.
- *
- * @param logSegmentPath
- * path to store log segment metadata
- * @return future of the retrieved log segment metadata
- */
- Future<LogSegmentMetadata> getLogSegment(String logSegmentPath);
-
- /**
- * Retrieve the list of log segments under <code>logSegmentsPath</code> and register a <i>listener</i>
- * for subsequent changes for the list of log segments.
- *
- * @param logSegmentsPath
- * path to store list of log segments
- * @param listener
- * log segment listener on log segment changes
- * @return future of the retrieved list of log segment names
- */
- Future<Versioned<List<String>>> getLogSegmentNames(String logSegmentsPath,
- LogSegmentNamesListener listener);
-
- /**
- * Unregister a log segment <code>listener</code> on log segment changes under <code>logSegmentsPath</code>.
- *
- * @param logSegmentsPath
- * log segments path
- * @param listener
- * log segment listener on log segment changes
- */
- void unregisterLogSegmentListener(String logSegmentsPath,
- LogSegmentNamesListener listener);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
deleted file mode 100644
index 70472ca..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
+++ /dev/null
@@ -1,47 +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.logsegment;
-
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.util.List;
-
-/**
- * An interface class to read entries {@link com.twitter.distributedlog.Entry}
- * from a random access log segment.
- */
-public interface LogSegmentRandomAccessEntryReader extends AsyncCloseable {
-
- /**
- * Read entries [startEntryId, endEntryId] from a random access log segment.
- *
- * @param startEntryId start entry id
- * @param endEntryId end entry id
- * @return A promise that when satisfied will contain a list of entries of [startEntryId, endEntryId].
- */
- Future<List<Entry.Reader>> readEntries(long startEntryId, long endEntryId);
-
- /**
- * Return the last add confirmed entry id (LAC).
- *
- * @return the last add confirmed entry id.
- */
- long getLastAddConfirmed();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentWriter.java
deleted file mode 100644
index a0b4610..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentWriter.java
+++ /dev/null
@@ -1,87 +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.logsegment;
-
-import com.google.common.annotations.Beta;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.io.AsyncAbortable;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.io.IOException;
-
-/**
- * An interface class to write log records into a log segment.
- */
-@Beta
-public interface LogSegmentWriter extends AsyncCloseable, AsyncAbortable {
-
- /**
- * Get the unique log segment id.
- *
- * @return log segment id.
- */
- public long getLogSegmentId();
-
- /**
- * Write a log record to a log segment.
- *
- * @param record single log record
- * @return a future representing write result. A {@link DLSN} is returned if write succeeds,
- * otherwise, exceptions are returned.
- * @throws com.twitter.distributedlog.exceptions.LogRecordTooLongException if log record is too long
- * @throws com.twitter.distributedlog.exceptions.InvalidEnvelopedEntryException on invalid enveloped entry
- * @throws LockingException if failed to acquire lock for the writer
- * @throws BKTransmitException if failed to transmit data to bk
- * @throws com.twitter.distributedlog.exceptions.WriteException if failed to write to bk
- */
- public Future<DLSN> asyncWrite(LogRecord record);
-
- /**
- * This isn't a simple synchronous version of {@code asyncWrite}. It has different semantic.
- * This method only writes data to the buffer and flushes buffer if needed.
- *
- * TODO: we should remove this method. when we rewrite synchronous writer based on asynchronous writer,
- * since this is the semantic needed to be provided in higher level but just calling write & flush.
- *
- * @param record single log record
- * @throws IOException when tried to flush the buffer.
- * @see LogSegmentWriter#asyncWrite(LogRecord)
- */
- public void write(LogRecord record) throws IOException;
-
- /**
- * Transmit the buffered data and wait for it being persisted and return the last acknowledged
- * transaction id.
- *
- * @return future representing the transmit result with last acknowledged transaction id.
- */
- public Future<Long> flush();
-
- /**
- * Commit the current acknowledged data. It is the consequent operation of {@link #flush()},
- * which makes all the acknowledged data visible to
- *
- * @return future representing the commit result.
- */
- public Future<Long> commit();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/PerStreamLogSegmentCache.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/PerStreamLogSegmentCache.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/PerStreamLogSegmentCache.java
deleted file mode 100644
index 5f88c5a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/PerStreamLogSegmentCache.java
+++ /dev/null
@@ -1,243 +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.logsegment;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Managing log segments in local cache.
- *
- * <p>
- * Caching of log segment metadata assumes that the data contained in the ZNodes for individual
- * log segments is never updated after creation i.e we never call setData. A log segment
- * is finalized by creating a new ZNode and deleting the in progress node. This code will have
- * to change if we change the behavior
- * </p>
- */
-public class PerStreamLogSegmentCache {
-
- static final Logger LOG = LoggerFactory.getLogger(PerStreamLogSegmentCache.class);
-
- protected final String streamName;
- protected final boolean validateLogSegmentSequenceNumber;
- protected final Map<String, LogSegmentMetadata> logSegments =
- new HashMap<String, LogSegmentMetadata>();
- protected final ConcurrentMap<Long, LogSegmentMetadata> lid2LogSegments =
- new ConcurrentHashMap<Long, LogSegmentMetadata>();
-
- @VisibleForTesting
- PerStreamLogSegmentCache(String streamName) {
- this(streamName, true);
- }
-
- public PerStreamLogSegmentCache(String streamName,
- boolean validateLogSegmentSequenceNumber) {
- this.streamName = streamName;
- this.validateLogSegmentSequenceNumber = validateLogSegmentSequenceNumber;
- }
-
- /**
- * Retrieve log segments from the cache.
- *
- * - first sort the log segments in ascending order
- * - do validation and assign corresponding sequence id
- * - apply comparator after validation
- *
- * @param comparator
- * comparator to sort the returned log segments.
- * @return list of sorted and filtered log segments.
- * @throws UnexpectedException if unexpected condition detected (e.g. ledger sequence number gap)
- */
- public List<LogSegmentMetadata> getLogSegments(Comparator<LogSegmentMetadata> comparator)
- throws UnexpectedException {
- List<LogSegmentMetadata> segmentsToReturn;
- synchronized (logSegments) {
- segmentsToReturn = new ArrayList<LogSegmentMetadata>(logSegments.size());
- segmentsToReturn.addAll(logSegments.values());
- }
- Collections.sort(segmentsToReturn, LogSegmentMetadata.COMPARATOR);
-
- LogSegmentMetadata prevSegment = null;
- if (validateLogSegmentSequenceNumber) {
- // validation ledger sequence number to ensure the log segments are unique.
- for (int i = 0; i < segmentsToReturn.size(); i++) {
- LogSegmentMetadata segment = segmentsToReturn.get(i);
-
- if (null != prevSegment
- && prevSegment.getVersion() >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
- && segment.getVersion() >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
- && prevSegment.getLogSegmentSequenceNumber() + 1 != segment.getLogSegmentSequenceNumber()) {
- LOG.error("{} found ledger sequence number gap between log segment {} and {}",
- new Object[] { streamName, prevSegment, segment });
- throw new UnexpectedException(streamName + " found ledger sequence number gap between log segment "
- + prevSegment.getLogSegmentSequenceNumber() + " and " + segment.getLogSegmentSequenceNumber());
- }
- prevSegment = segment;
- }
- }
-
- prevSegment = null;
- long startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
- for (int i = 0; i < segmentsToReturn.size(); i++) {
- LogSegmentMetadata segment = segmentsToReturn.get(i);
- // assign sequence id
- if (!segment.isInProgress()) {
- if (segment.supportsSequenceId()) {
- startSequenceId = segment.getStartSequenceId() + segment.getRecordCount();
- if (null != prevSegment && prevSegment.supportsSequenceId()
- && prevSegment.getStartSequenceId() > segment.getStartSequenceId()) {
- LOG.warn("{} found decreasing start sequence id in log segment {}, previous is {}",
- new Object[] { streamName, segment, prevSegment });
- }
- } else {
- startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
- }
- } else {
- if (segment.supportsSequenceId()) {
- LogSegmentMetadata newSegment = segment.mutator()
- .setStartSequenceId(startSequenceId == DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ? 0L : startSequenceId)
- .build();
- segmentsToReturn.set(i, newSegment);
- }
-
- break;
- }
- prevSegment = segment;
- }
- if (comparator != LogSegmentMetadata.COMPARATOR) {
- Collections.sort(segmentsToReturn, comparator);
- }
- return segmentsToReturn;
- }
-
- /**
- * Add the segment <i>metadata</i> for <i>name</i> in the cache.
- *
- * @param name
- * segment name.
- * @param metadata
- * segment metadata.
- */
- public void add(String name, LogSegmentMetadata metadata) {
- synchronized (logSegments) {
- if (!logSegments.containsKey(name)) {
- logSegments.put(name, metadata);
- LOG.info("{} added log segment ({} : {}) to cache.",
- new Object[]{ streamName, name, metadata });
- }
- LogSegmentMetadata oldMetadata = lid2LogSegments.remove(metadata.getLogSegmentId());
- if (null == oldMetadata) {
- lid2LogSegments.put(metadata.getLogSegmentId(), metadata);
- } else {
- if (oldMetadata.isInProgress() && !metadata.isInProgress()) {
- lid2LogSegments.put(metadata.getLogSegmentId(), metadata);
- } else {
- lid2LogSegments.put(oldMetadata.getLogSegmentId(), oldMetadata);
- }
- }
- }
- }
-
- /**
- * Retrieve log segment <code>name</code> from the cache.
- *
- * @param name
- * name of the log segment.
- * @return log segment metadata
- */
- public LogSegmentMetadata get(String name) {
- synchronized (logSegments) {
- return logSegments.get(name);
- }
- }
-
- /**
- * Update the log segment cache with removed/added segments.
- *
- * @param segmentsRemoved
- * segments that removed
- * @param segmentsAdded
- * segments that added
- */
- public void update(Set<String> segmentsRemoved,
- Map<String, LogSegmentMetadata> segmentsAdded) {
- synchronized (logSegments) {
- for (Map.Entry<String, LogSegmentMetadata> entry : segmentsAdded.entrySet()) {
- add(entry.getKey(), entry.getValue());
- }
- for (String segment : segmentsRemoved) {
- remove(segment);
- }
- }
- }
-
- /**
- * Diff with new received segment list <code>segmentReceived</code>.
- *
- * @param segmentsReceived
- * new received segment list
- * @return segments added (left) and removed (right).
- */
- public Pair<Set<String>, Set<String>> diff(Set<String> segmentsReceived) {
- Set<String> segmentsAdded;
- Set<String> segmentsRemoved;
- synchronized (logSegments) {
- Set<String> segmentsCached = logSegments.keySet();
- segmentsAdded = Sets.difference(segmentsReceived, segmentsCached).immutableCopy();
- segmentsRemoved = Sets.difference(segmentsCached, segmentsReceived).immutableCopy();
- }
- return Pair.of(segmentsAdded, segmentsRemoved);
- }
-
- /**
- * Remove log segment <code>name</code> from the cache.
- *
- * @param name
- * name of the log segment.
- * @return log segment metadata.
- */
- public LogSegmentMetadata remove(String name) {
- synchronized (logSegments) {
- LogSegmentMetadata metadata = logSegments.remove(name);
- if (null != metadata) {
- lid2LogSegments.remove(metadata.getLogSegmentId(), metadata);
- LOG.debug("Removed log segment ({} : {}) from cache.", name, metadata);
- }
- return metadata;
- }
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/RollingPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/RollingPolicy.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/RollingPolicy.java
deleted file mode 100644
index 0101bff..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/RollingPolicy.java
+++ /dev/null
@@ -1,33 +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.logsegment;
-
-import com.twitter.distributedlog.util.Sizable;
-
-public interface RollingPolicy {
- /**
- * Determines if a rollover may be appropriate at this time.
- *
- * @param sizable
- * Any object that is sizable.
- * @param lastRolloverTimeMs
- * last rolling time in millis.
- * @return true if a rollover is required. otherwise, false.
- */
- boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/SizeBasedRollingPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/SizeBasedRollingPolicy.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/SizeBasedRollingPolicy.java
deleted file mode 100644
index 8b1fa0f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/SizeBasedRollingPolicy.java
+++ /dev/null
@@ -1,34 +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.logsegment;
-
-import com.twitter.distributedlog.util.Sizable;
-
-public class SizeBasedRollingPolicy implements RollingPolicy {
-
- final long maxSize;
-
- public SizeBasedRollingPolicy(long maxSize) {
- this.maxSize = maxSize;
- }
-
- @Override
- public boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs) {
- return sizable.size() > maxSize;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/TimeBasedRollingPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/TimeBasedRollingPolicy.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/TimeBasedRollingPolicy.java
deleted file mode 100644
index bc88720..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/TimeBasedRollingPolicy.java
+++ /dev/null
@@ -1,46 +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.logsegment;
-
-import com.twitter.distributedlog.util.Sizable;
-import com.twitter.distributedlog.util.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TimeBasedRollingPolicy implements RollingPolicy {
-
- final static Logger LOG = LoggerFactory.getLogger(TimeBasedRollingPolicy.class);
-
- final long rollingIntervalMs;
-
- public TimeBasedRollingPolicy(long rollingIntervalMs) {
- this.rollingIntervalMs = rollingIntervalMs;
- }
-
- @Override
- public boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs) {
- long elapsedMs = Utils.elapsedMSec(lastRolloverTimeMs);
- boolean shouldSwitch = elapsedMs > rollingIntervalMs;
- if (shouldSwitch) {
- LOG.debug("Last Finalize Time: {} elapsed time (MSec): {}", lastRolloverTimeMs,
- elapsedMs);
- }
- return shouldSwitch;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/package-info.java
deleted file mode 100644
index 0f5b877..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Log Segment Management
- */
-package com.twitter.distributedlog.logsegment;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLConfig.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLConfig.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLConfig.java
deleted file mode 100644
index 178074a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLConfig.java
+++ /dev/null
@@ -1,39 +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.metadata;
-
-import java.io.IOException;
-
-/**
- * Specific config of a given implementation of DL
- */
-public interface DLConfig {
- /**
- * Serialize the dl config into a string.
- */
- public String serialize();
-
- /**
- * Deserialize the dl config from a readable stream.
- *
- * @param data
- * bytes to desrialize dl config.
- * @throws IOException if fail to deserialize the dl config string representation.
- */
- public void deserialize(byte[] data) throws IOException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
----------------------------------------------------------------------
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
deleted file mode 100644
index c0b5fb7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
+++ /dev/null
@@ -1,227 +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.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;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.StringReader;
-import java.net.URI;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Metadata of a given DL instance.
- */
-public class DLMetadata {
-
- static final Logger LOG = LoggerFactory.getLogger(DLMetadata.class);
-
- static final String LINE_SPLITTER = "\n";
- static final String BK_DL_TYPE = "BKDL";
- static final int METADATA_FORMAT_VERSION = 1;
-
- // metadata format version
- private int metadataFormatVersion = 0;
- // underlying dl type
- private String dlType;
- // underlying dl config
- private DLConfig dlConfig;
-
- public DLMetadata(String dlType, DLConfig dlConfig) {
- this(dlType, dlConfig, METADATA_FORMAT_VERSION);
- }
-
- DLMetadata(String dlType, DLConfig dlConfig, int metadataFormatVersion) {
- this.dlType = dlType;
- this.dlConfig = dlConfig;
- this.metadataFormatVersion = metadataFormatVersion;
- }
-
- /**
- * @return DL type
- */
- public String getDLType() {
- return dlType;
- }
-
- /**
- * @return DL Config
- */
- public DLConfig getDLConfig() {
- return dlConfig;
- }
-
- /**
- * Serialize the DL metadata into bytes array.
- *
- * @return bytes of DL metadata.
- */
- public byte[] serialize() {
- StringBuilder sb = new StringBuilder();
- sb.append(metadataFormatVersion).append(LINE_SPLITTER);
- sb.append(dlType).append(LINE_SPLITTER);
- sb.append(dlConfig.serialize());
- LOG.debug("Serialized dl metadata {}.", sb);
- return sb.toString().getBytes(UTF_8);
- }
-
- @Override
- public int hashCode() {
- return dlType.hashCode() * 13 + dlConfig.hashCode();
- }
-
- @Override
- public String toString() {
- return new String(serialize(), UTF_8);
- }
-
- public void update(URI uri) throws IOException {
- DistributedLogConfiguration conf = new DistributedLogConfiguration();
- ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .retryThreadCount(conf.getZKClientNumberRetryThreads())
- .requestRateLimit(conf.getZKRequestRateLimit())
- .zkAclId(conf.getZkAclId())
- .uri(uri)
- .build();
- byte[] data = serialize();
- try {
- zkc.get().setData(uri.getPath(), data, -1);
- } catch (KeeperException e) {
- throw new IOException("Fail to update dl metadata " + new String(data, UTF_8)
- + " to uri " + uri, e);
- } catch (InterruptedException e) {
- throw new IOException("Interrupted when updating dl metadata "
- + new String(data, UTF_8) + " to uri " + uri, e);
- } finally {
- zkc.close();
- }
- }
-
- @Override
- public boolean equals(Object o) {
- if (!(o instanceof DLMetadata)) {
- return false;
- }
- DLMetadata other = (DLMetadata) o;
- return dlType.equals(other.dlType) && dlConfig.equals(other.dlConfig);
- }
-
- public void create(URI uri) throws IOException {
- DistributedLogConfiguration conf = new DistributedLogConfiguration();
- ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .retryThreadCount(conf.getZKClientNumberRetryThreads())
- .requestRateLimit(conf.getZKRequestRateLimit())
- .zkAclId(conf.getZkAclId())
- .uri(uri)
- .build();
- byte[] data = serialize();
- try {
- Utils.zkCreateFullPathOptimistic(zkc, uri.getPath(), data,
- zkc.getDefaultACL(), CreateMode.PERSISTENT);
- } catch (KeeperException e) {
- throw new IOException("Fail to write dl metadata " + new String(data, UTF_8)
- + " to uri " + uri, e);
- } catch (InterruptedException e) {
- throw new IOException("Interrupted when writing dl metadata " + new String(data, UTF_8)
- + " to uri " + uri, e);
- } finally {
- zkc.close();
- }
- }
-
- public static void unbind(URI uri) throws IOException {
- DistributedLogConfiguration conf = new DistributedLogConfiguration();
- ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .retryThreadCount(conf.getZKClientNumberRetryThreads())
- .requestRateLimit(conf.getZKRequestRateLimit())
- .zkAclId(conf.getZkAclId())
- .uri(uri)
- .build();
- byte[] data = new byte[0];
- try {
- zkc.get().setData(uri.getPath(), data, -1);
- } catch (KeeperException ke) {
- throw new IOException("Fail to unbound dl metadata on uri " + uri, ke);
- } catch (InterruptedException ie) {
- throw new IOException("Interrupted when unbinding dl metadata on uri " + uri, ie);
- } finally {
- zkc.close();
- }
- }
-
- /**
- * Deserialize dl metadata of given <i>uri</i> from a given bytes array.
- *
- * @param uri
- * uri that stored dl metadata bindings
- * @param data
- * bytes of dl metadata
- * @return dl metadata
- * @throws IOException if failed to parse the bytes array
- */
- public static DLMetadata deserialize(URI uri, byte[] data) throws IOException {
- String metadata = new String(data, UTF_8);
- LOG.debug("Parsing dl metadata {}.", metadata);
- BufferedReader br = new BufferedReader(new StringReader(metadata));
- String versionLine = br.readLine();
- if (null == versionLine) {
- throw new IOException("Empty DL Metadata.");
- }
- int version;
- try {
- version = Integer.parseInt(versionLine);
- } catch (NumberFormatException nfe) {
- version = -1;
- }
- if (METADATA_FORMAT_VERSION != version) {
- throw new IOException("Metadata version not compatible. Expected "
- + METADATA_FORMAT_VERSION + " but got " + version);
- }
- String type = br.readLine();
- if (!BK_DL_TYPE.equals(type)) {
- throw new IOException("Invalid DL type : " + type);
- }
- DLConfig dlConfig = new BKDLConfig(uri);
- StringBuilder sb = new StringBuilder();
- String line;
- while (null != (line = br.readLine())) {
- sb.append(line);
- }
- dlConfig.deserialize(sb.toString().getBytes(UTF_8));
- return new DLMetadata(type, dlConfig, version);
- }
-
- public static DLMetadata create(BKDLConfig bkdlConfig) {
- return new DLMetadata(BK_DL_TYPE, bkdlConfig);
- }
-
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
deleted file mode 100644
index b2a417e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
+++ /dev/null
@@ -1,51 +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.metadata;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-
-public class DryrunLogSegmentMetadataStoreUpdater extends LogSegmentMetadataStoreUpdater {
-
- public DryrunLogSegmentMetadataStoreUpdater(DistributedLogConfiguration conf,
- LogSegmentMetadataStore metadataStore) {
- super(conf, metadataStore);
- }
-
- @Override
- public Transaction<Object> transaction() {
- return new Transaction<Object>() {
- @Override
- public void addOp(Op<Object> operation) {
- // no-op
- }
-
- @Override
- public Future<Void> execute() {
- return Future.Void();
- }
-
- @Override
- public void abort(Throwable reason) {
- // no-op
- }
- };
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadata.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadata.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadata.java
deleted file mode 100644
index c878d68..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadata.java
+++ /dev/null
@@ -1,175 +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.metadata;
-
-import java.net.URI;
-
-/**
- * Class to represent the layout and metadata of the zookeeper-based log metadata
- */
-public class LogMetadata {
-
- protected static String getLogComponentPath(URI uri, String logName, String logIdentifier, String component) {
- return String.format("%s/%s/%s%s", uri.getPath(), logName, logIdentifier, component);
- }
-
- /**
- * Get the top stream path for a given log.
- *
- * @param uri namespace to store the log
- * @param logName name of the log
- * @return top stream path
- */
- public static String getLogStreamPath(URI uri, String logName) {
- return String.format("%s/%s", uri.getPath(), logName);
- }
-
- /**
- * Get the log root path for a given log.
- *
- * @param uri
- * namespace to store the log
- * @param logName
- * name of the log
- * @param logIdentifier
- * identifier of the log
- * @return log root path
- */
- public static String getLogRootPath(URI uri, String logName, String logIdentifier) {
- return getLogComponentPath(uri, logName, logIdentifier, "");
- }
-
- /**
- * Get the logsegments root path for a given log.
- *
- * @param uri
- * namespace to store the log
- * @param logName
- * name of the log
- * @param logIdentifier
- * identifier of the log
- * @return logsegments root path
- */
- public static String getLogSegmentsPath(URI uri, String logName, String logIdentifier) {
- return getLogComponentPath(uri, logName, logIdentifier, LOGSEGMENTS_PATH);
- }
-
- public static final int LAYOUT_VERSION = -1;
- public final static String LOGSEGMENTS_PATH = "/ledgers";
- public final static String VERSION_PATH = "/version";
- // writer znodes
- public final static String MAX_TXID_PATH = "/maxtxid";
- public final static String LOCK_PATH = "/lock";
- public final static String ALLOCATION_PATH = "/allocation";
- // reader znodes
- public final static String READ_LOCK_PATH = "/readLock";
-
- protected final URI uri;
- protected final String logName;
- protected final String logIdentifier;
-
- // Root path of the log
- protected final String logRootPath;
- // Components
- protected final String logSegmentsPath;
- protected final String lockPath;
- protected final String maxTxIdPath;
- protected final String allocationPath;
-
- /**
- * metadata representation of a log
- *
- * @param uri
- * namespace to store the log
- * @param logName
- * name of the log
- * @param logIdentifier
- * identifier of the log
- */
- protected LogMetadata(URI uri,
- String logName,
- String logIdentifier) {
- this.uri = uri;
- this.logName = logName;
- this.logIdentifier = logIdentifier;
- this.logRootPath = getLogRootPath(uri, logName, logIdentifier);
- this.logSegmentsPath = logRootPath + LOGSEGMENTS_PATH;
- this.lockPath = logRootPath + LOCK_PATH;
- this.maxTxIdPath = logRootPath + MAX_TXID_PATH;
- this.allocationPath = logRootPath + ALLOCATION_PATH;
- }
-
- public URI getUri() {
- return uri;
- }
-
- public String getLogName() {
- return logName;
- }
-
- /**
- * Get the root path of the log.
- *
- * @return root path of the log.
- */
- public String getLogRootPath() {
- return logRootPath;
- }
-
- /**
- * Get the root path for log segments.
- *
- * @return root path for log segments
- */
- public String getLogSegmentsPath() {
- return this.logSegmentsPath;
- }
-
- /**
- * Get the path for a log segment of the log.
- *
- * @param segmentName
- * segment name
- * @return path for the log segment
- */
- public String getLogSegmentPath(String segmentName) {
- return this.logSegmentsPath + "/" + segmentName;
- }
-
- public String getLockPath() {
- return lockPath;
- }
-
- public String getMaxTxIdPath() {
- return maxTxIdPath;
- }
-
- public String getAllocationPath() {
- return allocationPath;
- }
-
- /**
- * Get the fully qualified name of the log.
- *
- * @return fully qualified name
- */
- public String getFullyQualifiedName() {
- return String.format("%s:%s", logName, logIdentifier);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForReader.java
deleted file mode 100644
index ff6bfca..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForReader.java
+++ /dev/null
@@ -1,103 +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.metadata;
-
-import com.google.common.base.Optional;
-
-import java.net.URI;
-
-/**
- * Log Metadata for Reader
- */
-public class LogMetadataForReader extends LogMetadata {
-
- /**
- * Get the root path to store subscription infos of a log.
- *
- * @param uri
- * namespace of the log
- * @param logName
- * name of the log
- * @param logIdentifier
- * identifier of the log
- * @return subscribers root path
- */
- public static String getSubscribersPath(URI uri, String logName, String logIdentifier) {
- return getLogComponentPath(uri, logName, logIdentifier, SUBSCRIBERS_PATH);
- }
-
- /**
- * Get the path that stores subscription info for a <code>subscriberId</code> for a <code>log</code>.
- *
- * @param uri
- * namespace of the log
- * @param logName
- * name of the log
- * @param logIdentifier
- * identifier of the log
- * @param subscriberId
- * subscriber id of the log
- * @return subscriber's path
- */
- public static String getSubscriberPath(URI uri, String logName, String logIdentifier, String subscriberId) {
- return String.format("%s/%s", getSubscribersPath(uri, logName, logIdentifier), subscriberId);
- }
-
- /**
- * Create a metadata representation of a log for reader.
- *
- * @param uri
- * namespace to store the log
- * @param logName
- * name of the log
- * @param logIdentifier
- * identifier of the log
- * @return metadata representation of a log for reader
- */
- public static LogMetadataForReader of(URI uri, String logName, String logIdentifier) {
- return new LogMetadataForReader(uri, logName, logIdentifier);
- }
-
- final static String SUBSCRIBERS_PATH = "/subscribers";
-
- /**
- * metadata representation of a log
- *
- * @param uri namespace to store the log
- * @param logName name of the log
- * @param logIdentifier identifier of the log
- */
- private LogMetadataForReader(URI uri, String logName, String logIdentifier) {
- super(uri, logName, logIdentifier);
- }
-
- /**
- * Get the readlock path for the log or a subscriber of the log.
- *
- * @param subscriberId
- * subscriber id. it is optional.
- * @return read lock path
- */
- public String getReadLockPath(Optional<String> subscriberId) {
- if (subscriberId.isPresent()) {
- return logRootPath + SUBSCRIBERS_PATH + "/" + subscriberId.get() + READ_LOCK_PATH;
- } else {
- return logRootPath + READ_LOCK_PATH;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForWriter.java
deleted file mode 100644
index 2284cbb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForWriter.java
+++ /dev/null
@@ -1,64 +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.metadata;
-
-import org.apache.bookkeeper.versioning.Versioned;
-
-import java.net.URI;
-
-/**
- * Log Metadata for writer
- */
-public class LogMetadataForWriter extends LogMetadata {
-
- private final Versioned<byte[]> maxLSSNData;
- private final Versioned<byte[]> maxTxIdData;
- private final Versioned<byte[]> allocationData;
-
- /**
- * metadata representation of a log
- *
- * @param uri namespace to store the log
- * @param logName name of the log
- * @param logIdentifier identifier of the log
- */
- public LogMetadataForWriter(URI uri,
- String logName,
- String logIdentifier,
- Versioned<byte[]> maxLSSNData,
- Versioned<byte[]> maxTxIdData,
- Versioned<byte[]> allocationData) {
- super(uri, logName, logIdentifier);
- this.maxLSSNData = maxLSSNData;
- this.maxTxIdData = maxTxIdData;
- this.allocationData = allocationData;
- }
-
- public Versioned<byte[]> getMaxLSSNData() {
- return maxLSSNData;
- }
-
- public Versioned<byte[]> getMaxTxIdData() {
- return maxTxIdData;
- }
-
- public Versioned<byte[]> getAllocationData() {
- return allocationData;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataStore.java
deleted file mode 100644
index 01dccb7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataStore.java
+++ /dev/null
@@ -1,66 +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.metadata;
-
-import com.google.common.annotations.Beta;
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.util.Future;
-
-import java.net.URI;
-import java.util.Iterator;
-
-/**
- * Interface for log metadata store.
- */
-@Beta
-public interface LogMetadataStore {
-
- /**
- * Create a stream and return it is namespace location.
- *
- * @param logName
- * name of the log
- * @return namespace location that stores this stream.
- */
- Future<URI> createLog(String logName);
-
- /**
- * Get the location of the log.
- *
- * @param logName
- * name of the log
- * @return namespace location that stores this stream.
- */
- Future<Optional<URI>> getLogLocation(String logName);
-
- /**
- * Retrieves logs from the namespace.
- *
- * @return iterator of logs of the namespace.
- */
- Future<Iterator<String>> getLogs();
-
- /**
- * Register a namespace listener on streams changes.
- *
- * @param listener
- * namespace listener
- */
- void registerNamespaceListener(NamespaceListener listener);
-}
[14/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/OrderedScheduler.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/OrderedScheduler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/OrderedScheduler.java
deleted file mode 100644
index 287bd6d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/OrderedScheduler.java
+++ /dev/null
@@ -1,490 +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.util;
-
-import com.google.common.base.Objects;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.stats.BroadCastStatsLogger;
-import com.twitter.util.ExecutorServiceFuturePool;
-import com.twitter.util.FuturePool;
-import com.twitter.util.Time;
-import com.twitter.util.Timer;
-import com.twitter.util.TimerTask;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.MathUtils;
-import scala.Function0;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-/**
- * Ordered Scheduler. It is thread pool based {@link ScheduledExecutorService}, additionally providing
- * the ability to execute/schedule tasks by <code>key</code>. Hence the tasks submitted by same <i>key</i>
- * will be executed in order.
- * <p>
- * The scheduler is comprised of multiple {@link MonitoredScheduledThreadPoolExecutor}s. Each
- * {@link MonitoredScheduledThreadPoolExecutor} is a single thread executor. Normal task submissions will
- * be submitted to executors in a random manner to guarantee load balancing. Keyed task submissions (e.g
- * {@link OrderedScheduler#apply(Object, Function0)} will be submitted to a dedicated executor based on
- * the hash value of submit <i>key</i>.
- *
- * <h3>Metrics</h3>
- *
- * <h4>Per Executor Metrics</h4>
- *
- * Metrics about individual executors are exposed via {@link Builder#perExecutorStatsLogger}
- * under <i>`scope`/`name`-executor-`id`-0</i>. `name` is the scheduler name provided by {@link Builder#name}
- * while `id` is the index of this executor in the pool. And corresponding stats of future pool of
- * that executor are exposed under <i>`scope`/`name`-executor-`id`-0/futurepool</i>.
- * <p>
- * See {@link MonitoredScheduledThreadPoolExecutor} and {@link MonitoredFuturePool} for per executor metrics
- * exposed.
- *
- * <h4>Aggregated Metrics</h4>
- * <ul>
- * <li>task_pending_time: opstats. measuring the characteristics about the time that tasks spent on
- * waiting being executed.
- * <li>task_execution_time: opstats. measuring the characteristics about the time that tasks spent on
- * executing.
- * <li>futurepool/task_pending_time: opstats. measuring the characteristics about the time that tasks spent
- * on waiting in future pool being executed.
- * <li>futurepool/task_execution_time: opstats. measuring the characteristics about the time that tasks spent
- * on executing.
- * <li>futurepool/task_enqueue_time: opstats. measuring the characteristics about the time that tasks spent on
- * submitting to future pool.
- * <li>futurepool/tasks_pending: gauge. how many tasks are pending in this future pool.
- * </ul>
- */
-public class OrderedScheduler implements ScheduledExecutorService {
-
- /**
- * Create a builder to build scheduler.
- *
- * @return scheduler builder
- */
- public static Builder newBuilder() {
- return new Builder();
- }
-
- /**
- * Builder for {@link OrderedScheduler}.
- */
- public static class Builder {
-
- private String name = "OrderedScheduler";
- private int corePoolSize = -1;
- private ThreadFactory threadFactory = null;
- private boolean traceTaskExecution = false;
- private long traceTaskExecutionWarnTimeUs = Long.MAX_VALUE;
- private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
- private StatsLogger perExecutorStatsLogger = NullStatsLogger.INSTANCE;
-
- /**
- * Set the name of this scheduler. It would be used as part of stats scope and thread name.
- *
- * @param name
- * name of the scheduler.
- * @return scheduler builder
- */
- public Builder name(String name) {
- this.name = name;
- return this;
- }
-
- /**
- * Set the number of threads to be used in this scheduler.
- *
- * @param corePoolSize the number of threads to keep in the pool, even
- * if they are idle
- * @return scheduler builder
- */
- public Builder corePoolSize(int corePoolSize) {
- this.corePoolSize = corePoolSize;
- return this;
- }
-
- /**
- * Set the thread factory that the scheduler uses to create a new thread.
- *
- * @param threadFactory the factory to use when the executor
- * creates a new thread
- * @return scheduler builder
- */
- public Builder threadFactory(ThreadFactory threadFactory) {
- this.threadFactory = threadFactory;
- return this;
- }
-
- /**
- * Enable/Disable exposing task execution stats.
- *
- * @param trace
- * flag to enable/disable exposing task execution stats.
- * @return scheduler builder
- */
- public Builder traceTaskExecution(boolean trace) {
- this.traceTaskExecution = trace;
- return this;
- }
-
- /**
- * Enable/Disable logging slow tasks whose execution time is above <code>timeUs</code>.
- *
- * @param timeUs
- * slow task execution time threshold in us.
- * @return scheduler builder.
- */
- public Builder traceTaskExecutionWarnTimeUs(long timeUs) {
- this.traceTaskExecutionWarnTimeUs = timeUs;
- return this;
- }
-
- /**
- * Expose the aggregated stats over <code>statsLogger</code>.
- *
- * @param statsLogger
- * stats logger to receive aggregated stats.
- * @return scheduler builder
- */
- public Builder statsLogger(StatsLogger statsLogger) {
- this.statsLogger = statsLogger;
- return this;
- }
-
- /**
- * Expose stats of individual executors over <code>perExecutorStatsLogger</code>.
- * Each executor's stats will be exposed under a sub-scope `name`-executor-`id`-0.
- * `name` is the scheduler name, while `id` is the index of the scheduler in the pool.
- *
- * @param perExecutorStatsLogger
- * stats logger to receive per executor stats.
- * @return scheduler builder
- */
- public Builder perExecutorStatsLogger(StatsLogger perExecutorStatsLogger) {
- this.perExecutorStatsLogger = perExecutorStatsLogger;
- return this;
- }
-
- /**
- * Build the ordered scheduler.
- *
- * @return ordered scheduler
- */
- public OrderedScheduler build() {
- if (corePoolSize <= 0) {
- corePoolSize = Runtime.getRuntime().availableProcessors();
- }
- if (null == threadFactory) {
- threadFactory = Executors.defaultThreadFactory();
- }
-
- return new OrderedScheduler(
- name,
- corePoolSize,
- threadFactory,
- traceTaskExecution,
- traceTaskExecutionWarnTimeUs,
- statsLogger,
- perExecutorStatsLogger);
- }
-
- }
-
- protected final String name;
- protected final int corePoolSize;
- protected final MonitoredScheduledThreadPoolExecutor[] executors;
- protected final MonitoredFuturePool[] futurePools;
- protected final Random random;
-
- private OrderedScheduler(String name,
- int corePoolSize,
- ThreadFactory threadFactory,
- boolean traceTaskExecution,
- long traceTaskExecutionWarnTimeUs,
- StatsLogger statsLogger,
- StatsLogger perExecutorStatsLogger) {
- this.name = name;
- this.corePoolSize = corePoolSize;
- this.executors = new MonitoredScheduledThreadPoolExecutor[corePoolSize];
- this.futurePools = new MonitoredFuturePool[corePoolSize];
- for (int i = 0; i < corePoolSize; i++) {
- ThreadFactory tf = new ThreadFactoryBuilder()
- .setNameFormat(name + "-executor-" + i + "-%d")
- .setThreadFactory(threadFactory)
- .build();
- StatsLogger broadcastStatsLogger =
- BroadCastStatsLogger.masterslave(perExecutorStatsLogger.scope("executor-" + i), statsLogger);
- executors[i] = new MonitoredScheduledThreadPoolExecutor(
- 1, tf, broadcastStatsLogger, traceTaskExecution);
- futurePools[i] = new MonitoredFuturePool(
- new ExecutorServiceFuturePool(executors[i]),
- broadcastStatsLogger.scope("futurepool"),
- traceTaskExecution,
- traceTaskExecutionWarnTimeUs);
- }
- this.random = new Random(System.currentTimeMillis());
- }
-
- protected MonitoredScheduledThreadPoolExecutor chooseExecutor() {
- return corePoolSize == 1 ? executors[0] : executors[random.nextInt(corePoolSize)];
- }
-
- protected MonitoredScheduledThreadPoolExecutor chooseExecutor(Object key) {
- return corePoolSize == 1 ? executors[0] :
- executors[MathUtils.signSafeMod(Objects.hashCode(key), corePoolSize)];
- }
-
- protected FuturePool chooseFuturePool(Object key) {
- return corePoolSize == 1 ? futurePools[0] :
- futurePools[MathUtils.signSafeMod(Objects.hashCode(key), corePoolSize)];
- }
-
- protected FuturePool chooseFuturePool() {
- return corePoolSize == 1 ? futurePools[0] : futurePools[random.nextInt(corePoolSize)];
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public ScheduledFuture<?> schedule(Runnable command, long delay, TimeUnit unit) {
- return chooseExecutor().schedule(command, delay, unit);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public <V> ScheduledFuture<V> schedule(Callable<V> callable, long delay, TimeUnit unit) {
- return chooseExecutor().schedule(callable, delay, unit);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public ScheduledFuture<?> scheduleAtFixedRate(Runnable command,
- long initialDelay, long period, TimeUnit unit) {
- return chooseExecutor().scheduleAtFixedRate(command, initialDelay, period, unit);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public ScheduledFuture<?> scheduleWithFixedDelay(Runnable command,
- long initialDelay, long delay, TimeUnit unit) {
- return chooseExecutor().scheduleWithFixedDelay(command, initialDelay, delay, unit);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void shutdown() {
- for (MonitoredScheduledThreadPoolExecutor executor : executors) {
- // Unregister gauges
- executor.unregisterGauges();
- executor.shutdown();
- }
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public List<Runnable> shutdownNow() {
- List<Runnable> runnables = new ArrayList<Runnable>();
- for (MonitoredScheduledThreadPoolExecutor executor : executors) {
- runnables.addAll(executor.shutdownNow());
- }
- return runnables;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public boolean isShutdown() {
- for (MonitoredScheduledThreadPoolExecutor executor : executors) {
- if (!executor.isShutdown()) {
- return false;
- }
- }
- return true;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public boolean isTerminated() {
- for (MonitoredScheduledThreadPoolExecutor executor : executors) {
- if (!executor.isTerminated()) {
- return false;
- }
- }
- return true;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public boolean awaitTermination(long timeout, TimeUnit unit)
- throws InterruptedException {
- for (MonitoredScheduledThreadPoolExecutor executor : executors) {
- if (!executor.awaitTermination(timeout, unit)) {
- return false;
- }
- }
- return true;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public <T> Future<T> submit(Callable<T> task) {
- return chooseExecutor().submit(task);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public <T> Future<T> submit(Runnable task, T result) {
- return chooseExecutor().submit(task, result);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public Future<?> submit(Runnable task) {
- return chooseExecutor().submit(task);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
- throws InterruptedException {
- return chooseExecutor().invokeAll(tasks);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit)
- throws InterruptedException {
- return chooseExecutor().invokeAll(tasks, timeout, unit);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public <T> T invokeAny(Collection<? extends Callable<T>> tasks)
- throws InterruptedException, ExecutionException {
- return chooseExecutor().invokeAny(tasks);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit)
- throws InterruptedException, ExecutionException, TimeoutException {
- return chooseExecutor().invokeAny(tasks, timeout, unit);
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void execute(Runnable command) {
- chooseExecutor().execute(command);
- }
-
- // Ordered Functions
-
- /**
- * Return a future pool used by <code>key</code>.
- *
- * @param key
- * key to order in the future pool
- * @return future pool
- */
- public FuturePool getFuturePool(Object key) {
- return chooseFuturePool(key);
- }
-
- /**
- * Execute the <code>function</code> in the executor that assigned by <code>key</code>.
- *
- * @see com.twitter.util.Future
- * @param key key of the <i>function</i> to run
- * @param function function to run
- * @return future representing the result of the <i>function</i>
- */
- public <T> com.twitter.util.Future<T> apply(Object key, Function0<T> function) {
- return chooseFuturePool(key).apply(function);
- }
-
- /**
- * Execute the <code>function</code> by the scheduler. It would be submitted to any executor randomly.
- *
- * @param function function to run
- * @return future representing the result of the <i>function</i>
- */
- public <T> com.twitter.util.Future<T> apply(Function0<T> function) {
- return chooseFuturePool().apply(function);
- }
-
- public ScheduledFuture<?> schedule(Object key, Runnable command, long delay, TimeUnit unit) {
- return chooseExecutor(key).schedule(command, delay, unit);
- }
-
- public ScheduledFuture<?> scheduleAtFixedRate(Object key,
- Runnable command,
- long initialDelay,
- long period,
- TimeUnit unit) {
- return chooseExecutor(key).scheduleAtFixedRate(command, initialDelay, period, unit);
- }
-
- public Future<?> submit(Object key, Runnable command) {
- return chooseExecutor(key).submit(command);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitLimiter.java
deleted file mode 100644
index 41c28a3..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitLimiter.java
+++ /dev/null
@@ -1,57 +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.util;
-
-/**
- * A simple limiter interface which tracks acquire/release of permits, for
- * example for tracking outstanding writes.
- */
-public interface PermitLimiter {
-
- public static PermitLimiter NULL_PERMIT_LIMITER = new PermitLimiter() {
- @Override
- public boolean acquire() {
- return true;
- }
- @Override
- public void release(int permits) {
- }
-
- @Override
- public void close() {
-
- }
- };
-
- /**
- * Acquire a permit.
- *
- * @return true if successfully acquire a permit, otherwise false.
- */
- boolean acquire();
-
- /**
- * Release a permit.
- */
- void release(int permits);
-
- /**
- * Close the resources created by the limiter
- */
- void close();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitManager.java
deleted file mode 100644
index 6a6d574..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitManager.java
+++ /dev/null
@@ -1,93 +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.util;
-
-public interface PermitManager {
-
- public static interface Permit {
- static final Permit ALLOWED = new Permit() {
- @Override
- public boolean isAllowed() {
- return true;
- }
- };
- boolean isAllowed();
- }
-
- public static PermitManager UNLIMITED_PERMIT_MANAGER = new PermitManager() {
- @Override
- public Permit acquirePermit() {
- return Permit.ALLOWED;
- }
-
- @Override
- public void releasePermit(Permit permit) {
- // nop
- }
-
- @Override
- public boolean allowObtainPermits() {
- return true;
- }
-
- @Override
- public boolean disallowObtainPermits(Permit permit) {
- return false;
- }
-
- @Override
- public void close() {
- // nop
- }
-
- };
-
- /**
- * Obetain a permit from permit manager.
- *
- * @return permit.
- */
- Permit acquirePermit();
-
- /**
- * Release a given permit.
- *
- * @param permit
- * permit to release
- */
- void releasePermit(Permit permit);
-
- /**
- * Allow obtaining permits.
- */
- boolean allowObtainPermits();
-
- /**
- * Disallow obtaining permits. Disallow needs to be performed under the context
- * of <i>permit</i>.
- *
- * @param permit
- * permit context to disallow
- */
- boolean disallowObtainPermits(Permit permit);
-
- /**
- * Release the resources
- */
- void close();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/RetryPolicyUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/RetryPolicyUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/RetryPolicyUtils.java
deleted file mode 100644
index 3565f98..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/RetryPolicyUtils.java
+++ /dev/null
@@ -1,45 +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.util;
-
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-
-/**
- * Utils for {@link org.apache.bookkeeper.zookeeper.RetryPolicy}
- */
-public class RetryPolicyUtils {
-
- /**
- * Infinite retry policy
- */
- public static final RetryPolicy DEFAULT_INFINITE_RETRY_POLICY = infiniteRetry(200, 2000);
-
- /**
- * Create an infinite retry policy with backoff time between <i>baseBackOffTimeMs</i> and
- * <i>maxBackoffTimeMs</i>.
- *
- * @param baseBackoffTimeMs base backoff time in milliseconds
- * @param maxBackoffTimeMs maximum backoff time in milliseconds
- * @return an infinite retry policy
- */
- public static RetryPolicy infiniteRetry(long baseBackoffTimeMs, long maxBackoffTimeMs) {
- return new BoundExponentialBackoffRetryPolicy(baseBackoffTimeMs, maxBackoffTimeMs, Integer.MAX_VALUE);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SafeQueueingFuturePool.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SafeQueueingFuturePool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SafeQueueingFuturePool.java
deleted file mode 100644
index d139a80..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SafeQueueingFuturePool.java
+++ /dev/null
@@ -1,115 +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.util;
-
-import com.google.common.base.Preconditions;
-
-import com.twitter.util.Function0;
-import com.twitter.util.FuturePool;
-import com.twitter.util.Future;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import scala.runtime.BoxedUnit;
-
-/**
- * Acts like a future pool, but collects failed apply calls into a queue to be applied
- * in-order on close. This happens either in the close thread or after close is called,
- * in the last operation to complete execution.
- * Ops submitted after close will not be scheduled, so its important to ensure no more
- * ops will be applied once close has been called.
- */
-public class SafeQueueingFuturePool<T> {
-
- static final Logger LOG = LoggerFactory.getLogger(SafeQueueingFuturePool.class);
-
- private boolean closed;
- private int outstanding;
- private ConcurrentLinkedQueue<Function0<T>> queue;
- private FuturePool orderedFuturePool;
-
- public SafeQueueingFuturePool(FuturePool orderedFuturePool) {
- this.closed = false;
- this.outstanding = 0;
- this.queue = new ConcurrentLinkedQueue<Function0<T>>();
- this.orderedFuturePool = orderedFuturePool;
- }
-
- public synchronized Future<T> apply(final Function0<T> fn) {
- Preconditions.checkNotNull(fn);
- if (closed) {
- return Future.exception(new RejectedExecutionException("Operation submitted to closed SafeQueueingFuturePool"));
- }
- ++outstanding;
- queue.add(fn);
- Future<T> result = orderedFuturePool.apply(new Function0<T>() {
- @Override
- public T apply() {
- return queue.poll().apply();
- }
- @Override
- public String toString() {
- return fn.toString();
- }
- }).ensure(new Function0<BoxedUnit>() {
- public BoxedUnit apply() {
- if (decrOutstandingAndCheckDone()) {
- applyAll();
- }
- return null;
- }
- });
- return result;
- }
-
- private synchronized boolean decrOutstandingAndCheckDone() {
- return --outstanding == 0 && closed;
- }
-
- public void close() {
- final boolean done;
- synchronized (this) {
- if (closed) {
- return;
- }
- closed = true;
- done = (outstanding == 0);
- }
- if (done) {
- applyAll();
- }
- }
-
- private void applyAll() {
- if (!queue.isEmpty()) {
- LOG.info("Applying {} items", queue.size());
- }
- while (!queue.isEmpty()) {
- queue.poll().apply();
- }
- }
-
- public synchronized int size() {
- return queue.size();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SchedulerUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SchedulerUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SchedulerUtils.java
deleted file mode 100644
index 9f756f0..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SchedulerUtils.java
+++ /dev/null
@@ -1,56 +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.util;
-
-import org.apache.bookkeeper.util.OrderedSafeExecutor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-public class SchedulerUtils {
-
- static final Logger logger = LoggerFactory.getLogger(SchedulerUtils.class);
-
- public static void shutdownScheduler(ExecutorService service, long timeout, TimeUnit timeUnit) {
- if (null == service) {
- return;
- }
- service.shutdown();
- try {
- service.awaitTermination(timeout, timeUnit);
- } catch (InterruptedException e) {
- logger.warn("Interrupted when shutting down scheduler : ", e);
- }
- service.shutdownNow();
- }
-
- public static void shutdownScheduler(OrderedSafeExecutor service, long timeout, TimeUnit timeUnit) {
- if (null == service) {
- return;
- }
- service.shutdown();
- try {
- service.awaitTermination(timeout, timeUnit);
- } catch (InterruptedException e) {
- logger.warn("Interrupted when shutting down scheduler : ", e);
- }
- service.forceShutdown(timeout, timeUnit);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sequencer.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sequencer.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sequencer.java
deleted file mode 100644
index 7ec50ba..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sequencer.java
+++ /dev/null
@@ -1,31 +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.util;
-
-/**
- * Sequencer generating transaction id.
- */
-public interface Sequencer {
-
- /**
- * Return next transaction id generated by the sequencer.
- *
- * @return next transaction id generated by the sequencer.
- */
- long nextId();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SimplePermitLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SimplePermitLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SimplePermitLimiter.java
deleted file mode 100644
index 4086a1e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SimplePermitLimiter.java
+++ /dev/null
@@ -1,117 +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.util;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Simple counter based {@link PermitLimiter}.
- *
- * <h3>Metrics</h3>
- * <ul>
- * <li> `permits`: gauge. how many permits are acquired right now?
- * <li> `permits`/*: opstats. the characteristics about number of permits already acquired on each acquires.
- * <li> `acquireFailure`: counter. how many acquires failed? failure means it already reached maximum permits
- * when trying to acquire.
- * </ul>
- */
-public class SimplePermitLimiter implements PermitLimiter {
-
- final Counter acquireFailureCounter;
- final OpStatsLogger permitsMetric;
- final AtomicInteger permits;
- final int permitsMax;
- final boolean darkmode;
- final Feature disableWriteLimitFeature;
- private StatsLogger statsLogger = null;
- private Gauge<Number> permitsGauge = null;
- private String permitsGaugeLabel = "";
-
- public SimplePermitLimiter(boolean darkmode, int permitsMax, StatsLogger statsLogger,
- boolean singleton, Feature disableWriteLimitFeature) {
- this.permits = new AtomicInteger(0);
- this.permitsMax = permitsMax;
- this.darkmode = darkmode;
- this.disableWriteLimitFeature = disableWriteLimitFeature;
-
- // stats
- if (singleton) {
- this.statsLogger = statsLogger;
- this.permitsGauge = new Gauge<Number>() {
- @Override
- public Number getDefaultValue() {
- return 0;
- }
- @Override
- public Number getSample() {
- return permits.get();
- }
- };
- this.permitsGaugeLabel = "permits";
- statsLogger.registerGauge(permitsGaugeLabel, permitsGauge);
- }
- acquireFailureCounter = statsLogger.getCounter("acquireFailure");
- permitsMetric = statsLogger.getOpStatsLogger("permits");
- }
-
- public boolean isDarkmode() {
- return darkmode || disableWriteLimitFeature.isAvailable();
- }
-
- @Override
- public boolean acquire() {
- permitsMetric.registerSuccessfulEvent(permits.get());
- if (permits.incrementAndGet() <= permitsMax || isDarkmode()) {
- return true;
- } else {
- acquireFailureCounter.inc();
- permits.decrementAndGet();
- return false;
- }
- }
-
- @Override
- public void release(int permitsToRelease) {
- permits.addAndGet(-permitsToRelease);
- }
-
- @Override
- public void close() {
- unregisterGauge();
- }
-
- @VisibleForTesting
- public int getPermits() {
- return permits.get();
- }
-
- public void unregisterGauge() {
- if (this.statsLogger != null && this.permitsGauge != null) {
- this.statsLogger.unregisterGauge(permitsGaugeLabel, permitsGauge);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sizable.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sizable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sizable.java
deleted file mode 100644
index 216d5ea..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sizable.java
+++ /dev/null
@@ -1,31 +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.util;
-
-/**
- * The {@code Sizable} interface is to provide the capability of calculating size
- * of any objects.
- */
-public interface Sizable {
- /**
- * Calculate the size for this instance.
- *
- * @return size of the instance.
- */
- long size();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/TimeSequencer.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/TimeSequencer.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/TimeSequencer.java
deleted file mode 100644
index 96e564e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/TimeSequencer.java
+++ /dev/null
@@ -1,39 +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.util;
-
-import com.twitter.distributedlog.DistributedLogConstants;
-
-/**
- * Time based sequencer. It generated non-decreasing transaction id using milliseconds.
- * It isn't thread-safe. The caller takes the responsibility on synchronization.
- */
-public class TimeSequencer implements Sequencer {
-
- private long lastId = DistributedLogConstants.INVALID_TXID;
-
- public void setLastId(long lastId) {
- this.lastId = lastId;
- }
-
- @Override
- public long nextId() {
- lastId = Math.max(lastId, System.currentTimeMillis());
- return lastId;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Transaction.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Transaction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Transaction.java
deleted file mode 100644
index 422bbda..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Transaction.java
+++ /dev/null
@@ -1,97 +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.util;
-
-import com.google.common.annotations.Beta;
-import com.twitter.util.Future;
-
-/**
- * Util class represents a transaction
- */
-@Beta
-public interface Transaction<OpResult> {
-
- /**
- * An operation executed in a transaction.
- */
- interface Op<OpResult> {
-
- /**
- * Execute after the transaction succeeds
- */
- void commit(OpResult r);
-
- /**
- * Execute after the transaction fails
- */
- void abort(Throwable t, OpResult r);
-
- }
-
- /**
- * Listener on the result of an {@link com.twitter.distributedlog.util.Transaction.Op}.
- *
- * @param <OpResult>
- */
- interface OpListener<OpResult> {
-
- /**
- * Trigger on operation committed.
- *
- * @param r
- * result to return
- */
- void onCommit(OpResult r);
-
- /**
- * Trigger on operation aborted.
- *
- * @param t
- * reason to abort
- */
- void onAbort(Throwable t);
- }
-
- /**
- * Add the operation to current transaction.
- *
- * @param operation
- * operation to execute under current transaction
- */
- void addOp(Op<OpResult> operation);
-
- /**
- * Execute the current transaction. If the transaction succeed, all operations will be
- * committed (via {@link com.twitter.distributedlog.util.Transaction.Op#commit(Object)}.
- * Otherwise, all operations will be aborted (via {@link Op#abort(Throwable, Object)}).
- *
- * @return future representing the result of transaction execution.
- */
- Future<Void> execute();
-
- /**
- * Abort current transaction. If this is called and the transaction haven't been executed by
- * {@link #execute()}, it would abort all operations. If the transaction has been executed,
- * the behavior is left up to implementation - if transaction is cancellable, the {@link #abort(Throwable)}
- * could attempt to cancel it.
- *
- * @param reason reason to abort the transaction
- */
- void abort(Throwable reason);
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Utils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Utils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Utils.java
deleted file mode 100644
index fce9bcd..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Utils.java
+++ /dev/null
@@ -1,607 +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.util;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicInteger;
-import javax.annotation.Nullable;
-
-import com.google.common.base.Objects;
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.google.common.io.Closeables;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.function.VoidFunctions;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
-
-/**
- * Basic Utilities.
- */
-public class Utils {
-
- private static final Logger logger = LoggerFactory.getLogger(Utils.class);
-
- /**
- * Current time from some arbitrary time base in the past, counting in
- * nanoseconds, and not affected by settimeofday or similar system clock
- * changes. This is appropriate to use when computing how much longer to
- * wait for an interval to expire.
- *
- * @return current time in nanoseconds.
- */
- public static long nowInNanos() {
- return System.nanoTime();
- }
-
- /**
- * Current time from some fixed base time - so useful for cross machine
- * comparison
- *
- * @return current time in milliseconds.
- */
- public static long nowInMillis() {
- return System.currentTimeMillis();
- }
-
- /**
- * Milliseconds elapsed since the time specified, the input is nanoTime
- * the only conversion happens when computing the elapsed time
- *
- * @param startMsecTime the start of the interval that we are measuring
- * @return elapsed time in milliseconds.
- */
- public static long elapsedMSec(long startMsecTime) {
- return (System.currentTimeMillis() - startMsecTime);
- }
-
- public static boolean randomPercent(double percent) {
- return (Math.random() * 100.0) <= percent;
- }
-
- /**
- * Synchronously create zookeeper path recursively and optimistically.
- *
- * @see #zkAsyncCreateFullPathOptimistic(ZooKeeperClient, String, byte[], List, CreateMode)
- * @param zkc Zookeeper client
- * @param path Zookeeper full path
- * @param data Zookeeper data
- * @param acl Acl of the zk path
- * @param createMode Create mode of zk path
- * @throws ZooKeeperClient.ZooKeeperConnectionException
- * @throws KeeperException
- * @throws InterruptedException
- */
- public static void zkCreateFullPathOptimistic(
- ZooKeeperClient zkc,
- String path,
- byte[] data,
- final List<ACL> acl,
- final CreateMode createMode)
- throws ZooKeeperClient.ZooKeeperConnectionException, KeeperException, InterruptedException {
- try {
- Await.result(zkAsyncCreateFullPathOptimistic(zkc, path, data, acl, createMode));
- } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
- throw zkce;
- } catch (KeeperException ke) {
- throw ke;
- } catch (InterruptedException ie) {
- throw ie;
- } catch (RuntimeException rte) {
- throw rte;
- } catch (Exception exc) {
- throw new RuntimeException("Unexpected Exception", exc);
- }
- }
-
- /**
- * Asynchronously create zookeeper path recursively and optimistically.
- *
- * @param zkc Zookeeper client
- * @param pathToCreate Zookeeper full path
- * @param parentPathShouldNotCreate The recursive creation should stop if this path doesn't exist
- * @param data Zookeeper data
- * @param acl Acl of the zk path
- * @param createMode Create mode of zk path
- * @param callback Callback
- * @param ctx Context object
- */
- public static void zkAsyncCreateFullPathOptimisticRecursive(
- final ZooKeeperClient zkc,
- final String pathToCreate,
- final Optional<String> parentPathShouldNotCreate,
- final byte[] data,
- final List<ACL> acl,
- final CreateMode createMode,
- final AsyncCallback.StringCallback callback,
- final Object ctx) {
- try {
- zkc.get().create(pathToCreate, data, acl, createMode, new AsyncCallback.StringCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, String name) {
-
- if (rc != KeeperException.Code.NONODE.intValue()) {
- callback.processResult(rc, path, ctx, name);
- return;
- }
-
- // Since we got a nonode, it means that my parents may not exist
- // ephemeral nodes can't have children so Create mode is always
- // persistent parents
- int lastSlash = pathToCreate.lastIndexOf('/');
- if (lastSlash <= 0) {
- callback.processResult(rc, path, ctx, name);
- return;
- }
- String parent = pathToCreate.substring(0, lastSlash);
- if (parentPathShouldNotCreate.isPresent() && Objects.equal(parentPathShouldNotCreate.get(), parent)) {
- // we should stop here
- callback.processResult(rc, path, ctx, name);
- return;
- }
- zkAsyncCreateFullPathOptimisticRecursive(zkc, parent, parentPathShouldNotCreate, new byte[0], acl,
- CreateMode.PERSISTENT, new AsyncCallback.StringCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, String name) {
- if (rc == KeeperException.Code.OK.intValue() || rc == KeeperException.Code.NODEEXISTS.intValue()) {
- // succeeded in creating the parent, now create the original path
- zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
- data, acl, createMode, callback, ctx);
- } else {
- callback.processResult(rc, path, ctx, name);
- }
- }
- }, ctx);
- }
- }, ctx);
- } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
- callback.processResult(DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE, zkce.getMessage(), ctx, pathToCreate);
- } catch (InterruptedException ie) {
- callback.processResult(DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE, ie.getMessage(), ctx, pathToCreate);
- }
- }
-
- /**
- * Asynchronously create zookeeper path recursively and optimistically.
- *
- * @param zkc Zookeeper client
- * @param pathToCreate Zookeeper full path
- * @param data Zookeeper data
- * @param acl Acl of the zk path
- * @param createMode Create mode of zk path
- */
- public static Future<BoxedUnit> zkAsyncCreateFullPathOptimistic(
- final ZooKeeperClient zkc,
- final String pathToCreate,
- final byte[] data,
- final List<ACL> acl,
- final CreateMode createMode) {
- Optional<String> parentPathShouldNotCreate = Optional.absent();
- return zkAsyncCreateFullPathOptimistic(
- zkc,
- pathToCreate,
- parentPathShouldNotCreate,
- data,
- acl,
- createMode);
- }
-
- /**
- * Asynchronously create zookeeper path recursively and optimistically
- *
- * @param zkc Zookeeper client
- * @param pathToCreate Zookeeper full path
- * @param parentPathShouldNotCreate zookeeper parent path should not be created
- * @param data Zookeeper data
- * @param acl Acl of the zk path
- * @param createMode Create mode of zk path
- */
- public static Future<BoxedUnit> zkAsyncCreateFullPathOptimistic(
- final ZooKeeperClient zkc,
- final String pathToCreate,
- final Optional<String> parentPathShouldNotCreate,
- final byte[] data,
- final List<ACL> acl,
- final CreateMode createMode) {
- final Promise<BoxedUnit> result = new Promise<BoxedUnit>();
-
- zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
- data, acl, createMode, new AsyncCallback.StringCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, String name) {
- handleKeeperExceptionCode(rc, path, result);
- }
- }, result);
-
- return result;
- }
-
- /**
- * Asynchronously create zookeeper path recursively and optimistically.
- *
- * @param zkc Zookeeper client
- * @param pathToCreate Zookeeper full path
- * @param data Zookeeper data
- * @param acl Acl of the zk path
- * @param createMode Create mode of zk path
- */
- public static Future<BoxedUnit> zkAsyncCreateFullPathOptimisticAndSetData(
- final ZooKeeperClient zkc,
- final String pathToCreate,
- final byte[] data,
- final List<ACL> acl,
- final CreateMode createMode) {
- final Promise<BoxedUnit> result = new Promise<BoxedUnit>();
-
- try {
- zkc.get().setData(pathToCreate, data, -1, new AsyncCallback.StatCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, Stat stat) {
- if (rc != KeeperException.Code.NONODE.intValue()) {
- handleKeeperExceptionCode(rc, path, result);
- return;
- }
-
- Optional<String> parentPathShouldNotCreate = Optional.absent();
- zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
- data, acl, createMode, new AsyncCallback.StringCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, String name) {
- handleKeeperExceptionCode(rc, path, result);
- }
- }, result);
- }
- }, result);
- } catch (Exception exc) {
- result.setException(exc);
- }
-
- return result;
- }
-
- private static void handleKeeperExceptionCode(int rc, String pathOrMessage, Promise<BoxedUnit> result) {
- if (KeeperException.Code.OK.intValue() == rc) {
- result.setValue(BoxedUnit.UNIT);
- } else if (DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE == rc) {
- result.setException(new ZooKeeperClient.ZooKeeperConnectionException(pathOrMessage));
- } else if (DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE == rc) {
- result.setException(new DLInterruptedException(pathOrMessage));
- } else {
- result.setException(KeeperException.create(KeeperException.Code.get(rc), pathOrMessage));
- }
- }
-
- public static Future<Versioned<byte[]>> zkGetData(ZooKeeperClient zkc, String path, boolean watch) {
- ZooKeeper zk;
- try {
- zk = zkc.get();
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- return Future.exception(FutureUtils.zkException(e, path));
- } catch (InterruptedException e) {
- return Future.exception(FutureUtils.zkException(e, path));
- }
- return zkGetData(zk, path, watch);
- }
-
- /**
- * Retrieve data from zookeeper <code>path</code>.
- *
- * @param path
- * zookeeper path to retrieve data
- * @param watch
- * whether to watch the path
- * @return future representing the versioned value. null version or null value means path doesn't exist.
- */
- public static Future<Versioned<byte[]>> zkGetData(ZooKeeper zk, String path, boolean watch) {
- final Promise<Versioned<byte[]>> promise = new Promise<Versioned<byte[]>>();
- zk.getData(path, watch, new AsyncCallback.DataCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
- if (KeeperException.Code.OK.intValue() == rc) {
- if (null == stat) {
- promise.setValue(new Versioned<byte[]>(null, null));
- } else {
- promise.setValue(new Versioned<byte[]>(data, new ZkVersion(stat.getVersion())));
- }
- } else if (KeeperException.Code.NONODE.intValue() == rc) {
- promise.setValue(new Versioned<byte[]>(null, null));
- } else {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
- }, null);
- return promise;
- }
-
- public static Future<ZkVersion> zkSetData(ZooKeeperClient zkc, String path, byte[] data, ZkVersion version) {
- ZooKeeper zk;
- try {
- zk = zkc.get();
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- return Future.exception(FutureUtils.zkException(e, path));
- } catch (InterruptedException e) {
- return Future.exception(FutureUtils.zkException(e, path));
- }
- return zkSetData(zk, path, data, version);
- }
-
- /**
- * Set <code>data</code> to zookeeper <code>path</code>.
- *
- * @param zk
- * zookeeper client
- * @param path
- * path to set data
- * @param data
- * data to set
- * @param version
- * version used to set data
- * @return future representing the version after this operation.
- */
- public static Future<ZkVersion> zkSetData(ZooKeeper zk, String path, byte[] data, ZkVersion version) {
- final Promise<ZkVersion> promise = new Promise<ZkVersion>();
- zk.setData(path, data, version.getZnodeVersion(), new AsyncCallback.StatCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, Stat stat) {
- if (KeeperException.Code.OK.intValue() == rc) {
- promise.updateIfEmpty(new Return<ZkVersion>(new ZkVersion(stat.getVersion())));
- return;
- }
- promise.updateIfEmpty(new Throw<ZkVersion>(
- KeeperException.create(KeeperException.Code.get(rc))));
- return;
- }
- }, null);
- return promise;
- }
-
- public static Future<Void> zkDelete(ZooKeeperClient zkc, String path, ZkVersion version) {
- ZooKeeper zk;
- try {
- zk = zkc.get();
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- return Future.exception(FutureUtils.zkException(e, path));
- } catch (InterruptedException e) {
- return Future.exception(FutureUtils.zkException(e, path));
- }
- return zkDelete(zk, path, version);
- }
-
- /**
- * Delete the given <i>path</i> from zookeeper.
- *
- * @param zk
- * zookeeper client
- * @param path
- * path to delete
- * @param version
- * version used to set data
- * @return future representing the version after this operation.
- */
- public static Future<Void> zkDelete(ZooKeeper zk, String path, ZkVersion version) {
- final Promise<Void> promise = new Promise<Void>();
- zk.delete(path, version.getZnodeVersion(), new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx) {
- if (KeeperException.Code.OK.intValue() == rc) {
- promise.updateIfEmpty(new Return<Void>(null));
- return;
- }
- promise.updateIfEmpty(new Throw<Void>(
- KeeperException.create(KeeperException.Code.get(rc))));
- return;
- }
- }, null);
- return promise;
- }
-
- /**
- * Delete the given <i>path</i> from zookeeper.
- *
- * @param zkc
- * zookeeper client
- * @param path
- * path to delete
- * @param version
- * version used to set data
- * @return future representing if the delete is successful. Return true if the node is deleted,
- * false if the node doesn't exist, otherwise future will throw exception
- *
- */
- public static Future<Boolean> zkDeleteIfNotExist(ZooKeeperClient zkc, String path, ZkVersion version) {
- ZooKeeper zk;
- try {
- zk = zkc.get();
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- return Future.exception(FutureUtils.zkException(e, path));
- } catch (InterruptedException e) {
- return Future.exception(FutureUtils.zkException(e, path));
- }
- final Promise<Boolean> promise = new Promise<Boolean>();
- zk.delete(path, version.getZnodeVersion(), new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx) {
- if (KeeperException.Code.OK.intValue() == rc ) {
- promise.setValue(true);
- } else if (KeeperException.Code.NONODE.intValue() == rc) {
- promise.setValue(false);
- } else {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
- }, null);
- return promise;
- }
-
- public static Future<Void> asyncClose(@Nullable AsyncCloseable closeable,
- boolean swallowIOException) {
- if (null == closeable) {
- return Future.Void();
- } else if (swallowIOException) {
- return FutureUtils.ignore(closeable.asyncClose());
- } else {
- return closeable.asyncClose();
- }
- }
-
- /**
- * Sync zookeeper client on given <i>path</i>.
- *
- * @param zkc
- * zookeeper client
- * @param path
- * path to sync
- * @return zookeeper client after sync
- * @throws IOException
- */
- public static ZooKeeper sync(ZooKeeperClient zkc, String path) throws IOException {
- ZooKeeper zk;
- try {
- zk = zkc.get();
- } catch (InterruptedException e) {
- throw new DLInterruptedException("Interrupted on checking if log " + path + " exists", e);
- }
- final CountDownLatch syncLatch = new CountDownLatch(1);
- final AtomicInteger syncResult = new AtomicInteger(0);
- zk.sync(path, new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx) {
- syncResult.set(rc);
- syncLatch.countDown();
- }
- }, null);
- try {
- syncLatch.await();
- } catch (InterruptedException e) {
- throw new DLInterruptedException("Interrupted on syncing zookeeper connection", e);
- }
- if (KeeperException.Code.OK.intValue() != syncResult.get()) {
- throw new ZKException("Error syncing zookeeper connection ",
- KeeperException.Code.get(syncResult.get()));
- }
- return zk;
- }
-
- /**
- * Close a closeable.
- *
- * @param closeable
- * closeable to close
- */
- public static void close(@Nullable Closeable closeable) {
- if (null == closeable) {
- return;
- }
- try {
- Closeables.close(closeable, true);
- } catch (IOException e) {
- // no-op. the exception is swallowed.
- }
- }
-
- /**
- * Close an async closeable.
- *
- * @param closeable
- * closeable to close
- */
- public static void close(@Nullable AsyncCloseable closeable)
- throws IOException {
- if (null == closeable) {
- return;
- }
- FutureUtils.result(closeable.asyncClose());
- }
-
- /**
- * Close an async closeable.
- *
- * @param closeable
- * closeable to close
- */
- public static void closeQuietly(@Nullable AsyncCloseable closeable) {
- if (null == closeable) {
- return;
- }
- try {
- FutureUtils.result(closeable.asyncClose());
- } catch (IOException e) {
- // no-op. the exception is swallowed.
- }
- }
-
- /**
- * Close the closeables in sequence.
- *
- * @param closeables
- * closeables to close
- * @return future represents the close future
- */
- public static Future<Void> closeSequence(ExecutorService executorService,
- AsyncCloseable... closeables) {
- return closeSequence(executorService, false, closeables);
- }
-
- /**
- * Close the closeables in sequence and ignore errors during closing.
- *
- * @param executorService executor to execute closeable
- * @param ignoreCloseError whether to ignore errors during closing
- * @param closeables list of closeables
- * @return future represents the close future.
- */
- public static Future<Void> closeSequence(ExecutorService executorService,
- boolean ignoreCloseError,
- AsyncCloseable... closeables) {
- List<AsyncCloseable> closeableList = Lists.newArrayListWithExpectedSize(closeables.length);
- for (AsyncCloseable closeable : closeables) {
- if (null == closeable) {
- closeableList.add(AsyncCloseable.NULL);
- } else {
- closeableList.add(closeable);
- }
- }
- return FutureUtils.processList(
- closeableList,
- ignoreCloseError ? AsyncCloseable.CLOSE_FUNC_IGNORE_ERRORS : AsyncCloseable.CLOSE_FUNC,
- executorService).map(VoidFunctions.LIST_TO_VOID_FUNC);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/package-info.java
deleted file mode 100644
index 193b814..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Utils
- */
-package com.twitter.distributedlog.util;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/DefaultZKOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/DefaultZKOp.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/DefaultZKOp.java
deleted file mode 100644
index 78292e9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/DefaultZKOp.java
+++ /dev/null
@@ -1,55 +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.zk;
-
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-
-import javax.annotation.Nullable;
-
-/**
- * Default zookeeper operation. No action on commiting or aborting.
- */
-public class DefaultZKOp extends ZKOp {
-
- public static DefaultZKOp of(Op op, OpListener<Void> listener) {
- return new DefaultZKOp(op, listener);
- }
-
- private final OpListener<Void> listener;
-
- private DefaultZKOp(Op op, @Nullable OpListener<Void> opListener) {
- super(op);
- this.listener = opListener;
- }
-
- @Override
- protected void commitOpResult(OpResult opResult) {
- if (null != listener) {
- listener.onCommit(null);
- }
- }
-
- @Override
- protected void abortOpResult(Throwable t, OpResult opResult) {
- if (null != listener) {
- listener.onAbort(t);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/LimitedPermitManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/LimitedPermitManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/LimitedPermitManager.java
deleted file mode 100644
index 78ff0a2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/LimitedPermitManager.java
+++ /dev/null
@@ -1,195 +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.zk;
-
-import com.twitter.distributedlog.util.PermitManager;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Manager to control all the log segments rolling.
- */
-public class LimitedPermitManager implements PermitManager, Runnable, Watcher {
-
- static final Logger LOG = LoggerFactory.getLogger(LimitedPermitManager.class);
-
- static enum PermitState {
- ALLOWED, DISALLOWED, DISABLED
- }
-
- class EpochPermit implements Permit {
-
- final PermitState state;
- final int epoch;
-
- EpochPermit(PermitState state) {
- this.state = state;
- this.epoch = LimitedPermitManager.this.epoch.get();
- }
-
- int getEpoch() {
- return epoch;
- }
-
- @Override
- public boolean isAllowed() {
- return PermitState.ALLOWED == state;
- }
- }
-
- boolean enablePermits = true;
- final Semaphore semaphore;
- final int period;
- final TimeUnit timeUnit;
- final ScheduledExecutorService executorService;
- final AtomicInteger epoch = new AtomicInteger(0);
- private StatsLogger statsLogger = null;
- private Gauge<Number> outstandingGauge = null;
-
- public LimitedPermitManager(int concurrency, int period, TimeUnit timeUnit,
- ScheduledExecutorService executorService) {
- this(concurrency, period, timeUnit, executorService, NullStatsLogger.INSTANCE);
- }
-
- public LimitedPermitManager(final int concurrency, int period, TimeUnit timeUnit,
- ScheduledExecutorService executorService, StatsLogger statsLogger) {
- if (concurrency > 0) {
- this.semaphore = new Semaphore(concurrency);
- } else {
- this.semaphore = null;
- }
- this.period = period;
- this.timeUnit = timeUnit;
- this.executorService = executorService;
- this.statsLogger = statsLogger;
- this.outstandingGauge = new Gauge<Number>() {
- @Override
- public Number getDefaultValue() {
- return 0;
- }
-
- @Override
- public Number getSample() {
- return null == semaphore ? 0 : concurrency - semaphore.availablePermits();
- }
- };
- this.statsLogger.scope("permits").registerGauge("outstanding", this.outstandingGauge);
- }
-
- @Override
- synchronized public Permit acquirePermit() {
- if (!enablePermits) {
- return new EpochPermit(PermitState.DISABLED);
- }
- if (null != semaphore) {
- return semaphore.tryAcquire() ? new EpochPermit(PermitState.ALLOWED) :
- new EpochPermit(PermitState.DISALLOWED);
- } else {
- return new EpochPermit(PermitState.ALLOWED);
- }
- }
-
- @Override
- synchronized public void releasePermit(Permit permit) {
- if (null != semaphore && permit.isAllowed()) {
- if (period <= 0) {
- semaphore.release();
- } else {
- try {
- executorService.schedule(this, period, timeUnit);
- } catch (RejectedExecutionException ree) {
- LOG.warn("Failed on scheduling releasing permit in given period ({}ms)." +
- " Release it immediately : ", timeUnit.toMillis(period), ree);
- semaphore.release();
- }
- }
- }
- }
-
- @Override
- synchronized public boolean disallowObtainPermits(Permit permit) {
- if (!(permit instanceof EpochPermit)) {
- return false;
- }
- if (epoch.getAndIncrement() == ((EpochPermit)permit).getEpoch()) {
- this.enablePermits = false;
- LOG.info("EnablePermits = {}, Epoch = {}.", this.enablePermits, epoch.get());
- return true;
- } else {
- return false;
- }
- }
-
- @Override
- public void close() {
- unregisterGauge();
- }
-
- @Override
- synchronized public boolean allowObtainPermits() {
- forceSetAllowPermits(true);
- return true;
- }
-
- synchronized void forceSetAllowPermits(boolean allowPermits) {
- epoch.getAndIncrement();
- this.enablePermits = allowPermits;
- LOG.info("EnablePermits = {}, Epoch = {}.", this.enablePermits, epoch.get());
- }
-
- @Override
- public void run() {
- semaphore.release();
- }
-
- @Override
- public void process(WatchedEvent event) {
- if (event.getType().equals(Event.EventType.None)) {
- switch (event.getState()) {
- case SyncConnected:
- forceSetAllowPermits(true);
- break;
- case Disconnected:
- forceSetAllowPermits(false);
- break;
- case Expired:
- forceSetAllowPermits(false);
- break;
- default:
- break;
- }
- }
- }
-
- public void unregisterGauge() {
- if(this.statsLogger != null && this.outstandingGauge != null) {
- this.statsLogger.scope("permits").unregisterGauge("outstanding", this.outstandingGauge);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKOp.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKOp.java
deleted file mode 100644
index 5675574..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKOp.java
+++ /dev/null
@@ -1,63 +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.zk;
-
-import com.twitter.distributedlog.util.Transaction;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-
-import javax.annotation.Nullable;
-
-/**
- * ZooKeeper Transaction Operation
- */
-public abstract class ZKOp implements Transaction.Op<Object> {
-
- protected final Op op;
-
- protected ZKOp(Op op) {
- this.op = op;
- }
-
- public Op getOp() {
- return op;
- }
-
- @Override
- public void commit(Object r) {
- assert(r instanceof OpResult);
- commitOpResult((OpResult) r);
- }
-
- protected abstract void commitOpResult(OpResult opResult);
-
- @Override
- public void abort(Throwable t, Object r) {
- assert(r instanceof OpResult);
- abortOpResult(t, (OpResult) r);
- }
-
- /**
- * Abort the operation with exception <i>t</i> and result <i>opResult</i>.
- *
- * @param t the reason to abort the operation
- * @param opResult the result of operation
- */
- protected abstract void abortOpResult(Throwable t,
- @Nullable OpResult opResult);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKTransaction.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKTransaction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKTransaction.java
deleted file mode 100644
index 57f9aa3..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKTransaction.java
+++ /dev/null
@@ -1,103 +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.zk;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.OpResult;
-
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-/**
- * ZooKeeper Transaction
- */
-public class ZKTransaction implements Transaction<Object>, AsyncCallback.MultiCallback {
-
- private final ZooKeeperClient zkc;
- private final List<ZKOp> ops;
- private final List<org.apache.zookeeper.Op> zkOps;
- private final Promise<Void> result;
- private final AtomicBoolean done = new AtomicBoolean(false);
-
- public ZKTransaction(ZooKeeperClient zkc) {
- this.zkc = zkc;
- this.ops = Lists.newArrayList();
- this.zkOps = Lists.newArrayList();
- this.result = new Promise<Void>();
- }
-
- @Override
- public void addOp(Op<Object> operation) {
- if (done.get()) {
- throw new IllegalStateException("Add an operation to a finished transaction");
- }
- assert(operation instanceof ZKOp);
- ZKOp zkOp = (ZKOp) operation;
- this.ops.add(zkOp);
- this.zkOps.add(zkOp.getOp());
- }
-
- @Override
- public Future<Void> execute() {
- if (!done.compareAndSet(false, true)) {
- return result;
- }
- try {
- zkc.get().multi(zkOps, this, result);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- result.setException(FutureUtils.zkException(e, ""));
- } catch (InterruptedException e) {
- result.setException(FutureUtils.zkException(e, ""));
- }
- return result;
- }
-
- @Override
- public void abort(Throwable cause) {
- if (!done.compareAndSet(false, true)) {
- return;
- }
- for (int i = 0; i < ops.size(); i++) {
- ops.get(i).abortOpResult(cause, null);
- }
- FutureUtils.setException(result, cause);
- }
-
- @Override
- public void processResult(int rc, String path, Object ctx, List<OpResult> results) {
- if (KeeperException.Code.OK.intValue() == rc) { // transaction succeed
- for (int i = 0; i < ops.size(); i++) {
- ops.get(i).commitOpResult(results.get(i));
- }
- FutureUtils.setValue(result, null);
- } else {
- KeeperException ke = KeeperException.create(KeeperException.Code.get(rc));
- for (int i = 0; i < ops.size(); i++) {
- ops.get(i).abortOpResult(ke, null != results ? results.get(i) : null);
- }
- FutureUtils.setException(result, ke);
- }
- }
-}
[02/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
new file mode 100644
index 0000000..e56a22d
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
@@ -0,0 +1,402 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.zk.ZKWatcherManager;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * ZooKeeper Client wrapper over {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}.
+ * It handles retries on session expires and provides a watcher manager {@link ZKWatcherManager}.
+ *
+ * <h3>Metrics</h3>
+ * <ul>
+ * <li> zookeeper operation stats are exposed under scope <code>zk</code> by
+ * {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}
+ * <li> stats on zookeeper watched events are exposed under scope <code>watcher</code> by
+ * {@link org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase}
+ * <li> stats about {@link ZKWatcherManager} are exposed under scope <code>watcher_manager</code>
+ * </ul>
+ */
+public class ZooKeeperClient {
+
+ public static interface Credentials {
+
+ Credentials NONE = new Credentials() {
+ @Override
+ public void authenticate(ZooKeeper zooKeeper) {
+ // noop
+ }
+ };
+
+ void authenticate(ZooKeeper zooKeeper);
+ }
+
+ public static class DigestCredentials implements Credentials {
+
+ String username;
+ String password;
+
+ public DigestCredentials(String username, String password) {
+ this.username = username;
+ this.password = password;
+ }
+
+ @Override
+ public void authenticate(ZooKeeper zooKeeper) {
+ zooKeeper.addAuthInfo("digest", String.format("%s:%s", username, password).getBytes(UTF_8));
+ }
+ }
+
+ public interface ZooKeeperSessionExpireNotifier {
+ void notifySessionExpired();
+ }
+
+ /**
+ * Indicates an error connecting to a zookeeper cluster.
+ */
+ public static class ZooKeeperConnectionException extends IOException {
+ private static final long serialVersionUID = 6682391687004819361L;
+
+ public ZooKeeperConnectionException(String message) {
+ super(message);
+ }
+
+ public ZooKeeperConnectionException(String message, Throwable cause) {
+ super(message, cause);
+ }
+ }
+
+ private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClient.class.getName());
+
+ private final String name;
+ private final int sessionTimeoutMs;
+ private final int defaultConnectionTimeoutMs;
+ private final String zooKeeperServers;
+ // GuardedBy "this", but still volatile for tests, where we want to be able to see writes
+ // made from within long synchronized blocks.
+ private volatile ZooKeeper zooKeeper = null;
+ private final RetryPolicy retryPolicy;
+ private final StatsLogger statsLogger;
+ private final int retryThreadCount;
+ private final double requestRateLimit;
+ private final Credentials credentials;
+ private volatile boolean authenticated = false;
+ private Stopwatch disconnectedStopwatch = null;
+
+ private boolean closed = false;
+
+ final Set<Watcher> watchers = new CopyOnWriteArraySet<Watcher>();
+
+ // watcher manager to manage watchers
+ private final ZKWatcherManager watcherManager;
+
+ /**
+ * Creates an unconnected client that will lazily attempt to connect on the first call to
+ * {@link #get}. All successful connections will be authenticated with the given
+ * {@code credentials}.
+ *
+ * @param sessionTimeoutMs
+ * ZK session timeout in milliseconds
+ * @param connectionTimeoutMs
+ * ZK connection timeout in milliseconds
+ * @param zooKeeperServers
+ * the set of servers forming the ZK cluster
+ */
+ ZooKeeperClient(int sessionTimeoutMs, int connectionTimeoutMs, String zooKeeperServers) {
+ this("default", sessionTimeoutMs, connectionTimeoutMs, zooKeeperServers, null, NullStatsLogger.INSTANCE, 1, 0,
+ Credentials.NONE);
+ }
+
+ ZooKeeperClient(String name,
+ int sessionTimeoutMs,
+ int connectionTimeoutMs,
+ String zooKeeperServers,
+ RetryPolicy retryPolicy,
+ StatsLogger statsLogger,
+ int retryThreadCount,
+ double requestRateLimit,
+ Credentials credentials) {
+ this.name = name;
+ this.sessionTimeoutMs = sessionTimeoutMs;
+ this.zooKeeperServers = zooKeeperServers;
+ this.defaultConnectionTimeoutMs = connectionTimeoutMs;
+ this.retryPolicy = retryPolicy;
+ this.statsLogger = statsLogger;
+ this.retryThreadCount = retryThreadCount;
+ this.requestRateLimit = requestRateLimit;
+ this.credentials = credentials;
+ this.watcherManager = ZKWatcherManager.newBuilder()
+ .name(name)
+ .zkc(this)
+ .statsLogger(statsLogger.scope("watcher_manager"))
+ .build();
+ }
+
+ public List<ACL> getDefaultACL() {
+ if (Credentials.NONE == credentials) {
+ return ZooDefs.Ids.OPEN_ACL_UNSAFE;
+ } else {
+ return DistributedLogConstants.EVERYONE_READ_CREATOR_ALL;
+ }
+ }
+
+ public ZKWatcherManager getWatcherManager() {
+ return watcherManager;
+ }
+
+ /**
+ * Returns the current active ZK connection or establishes a new one if none has yet been
+ * established or a previous connection was disconnected or had its session time out.
+ *
+ * @return a connected ZooKeeper client
+ * @throws ZooKeeperConnectionException if there was a problem connecting to the ZK cluster
+ * @throws InterruptedException if interrupted while waiting for a connection to be established
+ * @throws TimeoutException if a connection could not be established within the configured
+ * session timeout
+ */
+ public synchronized ZooKeeper get()
+ throws ZooKeeperConnectionException, InterruptedException {
+
+ try {
+ FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
+ } catch (IOException ioe) {
+ throw new ZooKeeperConnectionException("Client " + name + " failed on establishing zookeeper connection", ioe);
+ }
+
+ // This indicates that the client was explictly closed
+ if (closed) {
+ throw new ZooKeeperConnectionException("Client " + name + " has already been closed");
+ }
+
+ // the underneath zookeeper is retryable zookeeper
+ if (zooKeeper != null && retryPolicy != null) {
+ if (zooKeeper.getState().equals(ZooKeeper.States.CONNECTED)) {
+ // the zookeeper client is connected
+ disconnectedStopwatch = null;
+ } else {
+ if (disconnectedStopwatch == null) {
+ disconnectedStopwatch = Stopwatch.createStarted();
+ } else {
+ long disconnectedMs = disconnectedStopwatch.elapsed(TimeUnit.MILLISECONDS);
+ if (disconnectedMs > defaultConnectionTimeoutMs) {
+ closeInternal();
+ authenticated = false;
+ }
+ }
+ }
+ }
+
+ if (zooKeeper == null) {
+ zooKeeper = buildZooKeeper();
+ disconnectedStopwatch = null;
+ }
+
+ // In case authenticate throws an exception, the caller can try to recover the client by
+ // calling get again.
+ if (!authenticated) {
+ credentials.authenticate(zooKeeper);
+ authenticated = true;
+ }
+
+ return zooKeeper;
+ }
+
+ private ZooKeeper buildZooKeeper()
+ throws ZooKeeperConnectionException, InterruptedException {
+ Watcher watcher = new Watcher() {
+ @Override
+ public void process(WatchedEvent event) {
+ switch (event.getType()) {
+ case None:
+ switch (event.getState()) {
+ case Expired:
+ if (null == retryPolicy) {
+ LOG.info("ZooKeeper {}' session expired. Event: {}", name, event);
+ closeInternal();
+ }
+ authenticated = false;
+ break;
+ case Disconnected:
+ if (null == retryPolicy) {
+ LOG.info("ZooKeeper {} is disconnected from zookeeper now," +
+ " but it is OK unless we received EXPIRED event.", name);
+ }
+ // Mark as not authenticated if expired or disconnected. In both cases
+ // we lose any attached auth info. Relying on Expired/Disconnected is
+ // sufficient since all Expired/Disconnected events are processed before
+ // all SyncConnected events, and the underlying member is not updated until
+ // SyncConnected is received.
+ authenticated = false;
+ break;
+ default:
+ break;
+ }
+ }
+
+ try {
+ for (Watcher watcher : watchers) {
+ try {
+ watcher.process(event);
+ } catch (Throwable t) {
+ LOG.warn("Encountered unexpected exception from watcher {} : ", watcher, t);
+ }
+ }
+ } catch (Throwable t) {
+ LOG.warn("Encountered unexpected exception when firing watched event {} : ", event, t);
+ }
+ }
+ };
+
+ Set<Watcher> watchers = new HashSet<Watcher>();
+ watchers.add(watcher);
+
+ ZooKeeper zk;
+ try {
+ RetryPolicy opRetryPolicy = null == retryPolicy ?
+ new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) : retryPolicy;
+ RetryPolicy connectRetryPolicy = null == retryPolicy ?
+ new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) :
+ new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, Integer.MAX_VALUE);
+ zk = org.apache.bookkeeper.zookeeper.ZooKeeperClient.newBuilder()
+ .connectString(zooKeeperServers)
+ .sessionTimeoutMs(sessionTimeoutMs)
+ .watchers(watchers)
+ .operationRetryPolicy(opRetryPolicy)
+ .connectRetryPolicy(connectRetryPolicy)
+ .statsLogger(statsLogger)
+ .retryThreadCount(retryThreadCount)
+ .requestRateLimit(requestRateLimit)
+ .build();
+ } catch (KeeperException e) {
+ throw new ZooKeeperConnectionException("Problem connecting to servers: " + zooKeeperServers, e);
+ } catch (IOException e) {
+ throw new ZooKeeperConnectionException("Problem connecting to servers: " + zooKeeperServers, e);
+ }
+ return zk;
+ }
+
+ /**
+ * Clients that need to re-establish state after session expiration can register an
+ * {@code onExpired} command to execute.
+ *
+ * @param onExpired the {@code Command} to register
+ * @return the new {@link Watcher} which can later be passed to {@link #unregister} for
+ * removal.
+ */
+ public Watcher registerExpirationHandler(final ZooKeeperSessionExpireNotifier onExpired) {
+ Watcher watcher = new Watcher() {
+ @Override
+ public void process(WatchedEvent event) {
+ if (event.getType() == EventType.None && event.getState() == KeeperState.Expired) {
+ try {
+ onExpired.notifySessionExpired();
+ } catch (Exception exc) {
+ // do nothing
+ }
+ }
+ }
+ };
+ register(watcher);
+ return watcher;
+ }
+
+ /**
+ * Clients that need to register a top-level {@code Watcher} should do so using this method. The
+ * registered {@code watcher} will remain registered across re-connects and session expiration
+ * events.
+ *
+ * @param watcher the {@code Watcher to register}
+ */
+ public void register(Watcher watcher) {
+ if (null != watcher) {
+ watchers.add(watcher);
+ }
+ }
+
+ /**
+ * Clients can attempt to unregister a top-level {@code Watcher} that has previously been
+ * registered.
+ *
+ * @param watcher the {@code Watcher} to unregister as a top-level, persistent watch
+ * @return whether the given {@code Watcher} was found and removed from the active set
+ */
+ public boolean unregister(Watcher watcher) {
+ return null != watcher && watchers.remove(watcher);
+ }
+
+ /**
+ * Closes the current connection if any expiring the current ZooKeeper session. Any subsequent
+ * calls to this method will no-op until the next successful {@link #get}.
+ */
+ public synchronized void closeInternal() {
+ if (zooKeeper != null) {
+ try {
+ LOG.info("Closing zookeeper client {}.", name);
+ zooKeeper.close();
+ LOG.info("Closed zookeeper client {}.", name);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ LOG.warn("Interrupted trying to close zooKeeper {} : ", name, e);
+ } finally {
+ zooKeeper = null;
+ }
+ }
+ }
+
+ /**
+ * Closes the the underlying zookeeper instance.
+ * Subsequent attempts to {@link #get} will fail
+ */
+ public synchronized void close() {
+ if (closed) {
+ return;
+ }
+ LOG.info("Close zookeeper client {}.", name);
+ closeInternal();
+ // unregister gauges to prevent GC spiral
+ this.watcherManager.unregisterGauges();
+ closed = true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
new file mode 100644
index 0000000..0c200ce
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
@@ -0,0 +1,233 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Preconditions;
+import org.apache.distributedlog.ZooKeeperClient.Credentials;
+import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+
+/**
+ * Builder to build zookeeper client.
+ */
+public class ZooKeeperClientBuilder {
+
+ static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClientBuilder.class);
+
+ /**
+ * Create a zookeeper client builder to build zookeeper clients.
+ *
+ * @return zookeeper client builder.
+ */
+ public static ZooKeeperClientBuilder newBuilder() {
+ return new ZooKeeperClientBuilder();
+ }
+
+ // name
+ private String name = "default";
+ // sessionTimeoutMs
+ private int sessionTimeoutMs = -1;
+ // conectionTimeoutMs
+ private int conectionTimeoutMs = -1;
+ // zkServers
+ private String zkServers = null;
+ // retry policy
+ private RetryPolicy retryPolicy = null;
+ // stats logger
+ private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+ // retry executor thread count
+ private int retryThreadCount = 1;
+ // zookeeper access requestRateLimit limit
+ private double requestRateLimit = 0;
+ // Did call the zkAclId setter on the builder, used to ensure the setter is set.
+ private boolean zkAclIdSet = false;
+ private String zkAclId;
+
+ // Cached ZooKeeper Client
+ private ZooKeeperClient cachedClient = null;
+
+ private ZooKeeperClientBuilder() {}
+
+ /**
+ * Set zookeeper client name
+ *
+ * @param name zookeeper client name
+ * @return zookeeper client builder
+ */
+ public synchronized ZooKeeperClientBuilder name(String name) {
+ this.name = name;
+ return this;
+ }
+
+ /**
+ * Set zookeeper session timeout in milliseconds.
+ *
+ * @param sessionTimeoutMs
+ * session timeout in milliseconds.
+ * @return zookeeper client builder.
+ */
+ public synchronized ZooKeeperClientBuilder sessionTimeoutMs(int sessionTimeoutMs) {
+ this.sessionTimeoutMs = sessionTimeoutMs;
+ if (this.conectionTimeoutMs <= 0) {
+ this.conectionTimeoutMs = 2 * sessionTimeoutMs;
+ }
+ return this;
+ }
+
+ public synchronized ZooKeeperClientBuilder retryThreadCount(int retryThreadCount) {
+ this.retryThreadCount = retryThreadCount;
+ return this;
+ }
+
+ public synchronized ZooKeeperClientBuilder requestRateLimit(double requestRateLimit) {
+ this.requestRateLimit = requestRateLimit;
+ return this;
+ }
+
+ /**
+ * Set zookeeper connection timeout in milliseconds
+ *
+ * @param connectionTimeoutMs
+ * connection timeout ms.
+ * @return builder
+ */
+ public synchronized ZooKeeperClientBuilder connectionTimeoutMs(int connectionTimeoutMs) {
+ this.conectionTimeoutMs = connectionTimeoutMs;
+ return this;
+ }
+
+ /**
+ * Set ZooKeeper Connect String.
+ *
+ * @param zkServers
+ * zookeeper servers to connect.
+ * @return builder
+ */
+ public synchronized ZooKeeperClientBuilder zkServers(String zkServers) {
+ this.zkServers = zkServers;
+ return this;
+ }
+
+ /**
+ * Set DistributedLog URI.
+ *
+ * @param uri
+ * distributedlog uri.
+ * @return builder.
+ */
+ public synchronized ZooKeeperClientBuilder uri(URI uri) {
+ this.zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
+ return this;
+ }
+
+ /**
+ * Build zookeeper client using existing <i>zkc</i> client.
+ *
+ * @param zkc
+ * zookeeper client.
+ * @return builder
+ */
+ public synchronized ZooKeeperClientBuilder zkc(ZooKeeperClient zkc) {
+ this.cachedClient = zkc;
+ return this;
+ }
+
+ /**
+ * Build zookeeper client with given retry policy <i>retryPolicy</i>.
+ *
+ * @param retryPolicy
+ * retry policy
+ * @return builder
+ */
+ public synchronized ZooKeeperClientBuilder retryPolicy(RetryPolicy retryPolicy) {
+ this.retryPolicy = retryPolicy;
+ return this;
+ }
+
+ /**
+ * Build zookeeper client with given stats logger <i>statsLogger</i>.
+ *
+ * @param statsLogger
+ * stats logger to expose zookeeper stats
+ * @return builder
+ */
+ public synchronized ZooKeeperClientBuilder statsLogger(StatsLogger statsLogger) {
+ this.statsLogger = statsLogger;
+ return this;
+ }
+
+ /**
+ * * Build zookeeper client with given zk acl digest id <i>zkAclId</i>.
+ */
+ public synchronized ZooKeeperClientBuilder zkAclId(String zkAclId) {
+ this.zkAclIdSet = true;
+ this.zkAclId = zkAclId;
+ return this;
+ }
+
+ private void validateParameters() {
+ Preconditions.checkNotNull(zkServers, "No zk servers provided.");
+ Preconditions.checkArgument(conectionTimeoutMs > 0,
+ "Invalid connection timeout : %d", conectionTimeoutMs);
+ Preconditions.checkArgument(sessionTimeoutMs > 0,
+ "Invalid session timeout : %d", sessionTimeoutMs);
+ Preconditions.checkNotNull(statsLogger, "No stats logger provided.");
+ Preconditions.checkArgument(zkAclIdSet, "Zookeeper acl id not set.");
+ }
+
+ /**
+ * Build a zookeeper client.
+ *
+ * @return zookeeper client.
+ */
+ public synchronized ZooKeeperClient build() {
+ if (null == cachedClient) {
+ cachedClient = buildClient();
+ }
+ return cachedClient;
+ }
+
+ private ZooKeeperClient buildClient() {
+ validateParameters();
+
+ Credentials credentials = Credentials.NONE;
+ if (null != zkAclId) {
+ credentials = new DigestCredentials(zkAclId, zkAclId);
+ }
+
+ return new ZooKeeperClient(
+ name,
+ sessionTimeoutMs,
+ conectionTimeoutMs,
+ zkServers,
+ retryPolicy,
+ statsLogger,
+ retryThreadCount,
+ requestRateLimit,
+ credentials
+ );
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java
new file mode 100644
index 0000000..2c3e738
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java
@@ -0,0 +1,74 @@
+/**
+ * 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 org.apache.distributedlog.acl;
+
+/**
+ * Access Control on stream operations
+ */
+public interface AccessControlManager {
+
+ /**
+ * Whether allowing writing to a stream.
+ *
+ * @param stream
+ * Stream to write
+ * @return true if allowing writing to the given stream, otherwise false.
+ */
+ boolean allowWrite(String stream);
+
+ /**
+ * Whether allowing truncating a given stream.
+ *
+ * @param stream
+ * Stream to truncate
+ * @return true if allowing truncating a given stream.
+ */
+ boolean allowTruncate(String stream);
+
+ /**
+ * Whether allowing deleting a given stream.
+ *
+ * @param stream
+ * Stream to delete
+ * @return true if allowing deleting a given stream.
+ */
+ boolean allowDelete(String stream);
+
+ /**
+ * Whether allowing proxies to acquire a given stream.
+ *
+ * @param stream
+ * stream to acquire
+ * @return true if allowing proxies to acquire the given stream.
+ */
+ boolean allowAcquire(String stream);
+
+ /**
+ * Whether allowing proxies to release ownership for a given stream.
+ *
+ * @param stream
+ * stream to release
+ * @return true if allowing proxies to release a given stream.
+ */
+ boolean allowRelease(String stream);
+
+ /**
+ * Close the access control manager.
+ */
+ void close();
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java
new file mode 100644
index 0000000..bf3352a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java
@@ -0,0 +1,55 @@
+/**
+ * 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 org.apache.distributedlog.acl;
+
+public class DefaultAccessControlManager implements AccessControlManager {
+
+ public static final DefaultAccessControlManager INSTANCE = new DefaultAccessControlManager();
+
+ private DefaultAccessControlManager() {
+ }
+
+ @Override
+ public boolean allowWrite(String stream) {
+ return true;
+ }
+
+ @Override
+ public boolean allowTruncate(String stream) {
+ return true;
+ }
+
+ @Override
+ public boolean allowDelete(String stream) {
+ return true;
+ }
+
+ @Override
+ public boolean allowAcquire(String stream) {
+ return true;
+ }
+
+ @Override
+ public boolean allowRelease(String stream) {
+ return true;
+ }
+
+ @Override
+ public void close() {
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/acl/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/package-info.java
new file mode 100644
index 0000000..4218bfc
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Access Control for distributedlog streams.
+ */
+package org.apache.distributedlog.acl;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
new file mode 100644
index 0000000..4e94984
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
@@ -0,0 +1,921 @@
+/**
+ * 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 org.apache.distributedlog.admin;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.ReadUtils;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.impl.acl.ZKAccessControl;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.impl.federated.FederatedZKLogMetadataStore;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
+import org.apache.distributedlog.metadata.MetadataUpdater;
+import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.thrift.AccessControlEntry;
+import org.apache.distributedlog.tools.DistributedLogTool;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.SchedulerUtils;
+import com.twitter.util.Await;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+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;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Admin Tool for DistributedLog.
+ */
+public class DistributedLogAdmin extends DistributedLogTool {
+
+ static final Logger LOG = LoggerFactory.getLogger(DistributedLogAdmin.class);
+
+ /**
+ * Fix inprogress segment with lower ledger sequence number.
+ *
+ * @param namespace
+ * dl namespace
+ * @param metadataUpdater
+ * metadata updater.
+ * @param streamName
+ * stream name.
+ * @param verbose
+ * print verbose messages.
+ * @param interactive
+ * is confirmation needed before executing actual action.
+ * @throws IOException
+ */
+ public static void fixInprogressSegmentWithLowerSequenceNumber(final DistributedLogNamespace namespace,
+ final MetadataUpdater metadataUpdater,
+ final String streamName,
+ final boolean verbose,
+ final boolean interactive) throws IOException {
+ DistributedLogManager dlm = namespace.openLog(streamName);
+ try {
+ List<LogSegmentMetadata> segments = dlm.getLogSegments();
+ if (verbose) {
+ System.out.println("LogSegments for " + streamName + " : ");
+ for (LogSegmentMetadata segment : segments) {
+ System.out.println(segment.getLogSegmentSequenceNumber() + "\t: " + segment);
+ }
+ }
+ LOG.info("Get log segments for {} : {}", streamName, segments);
+ // validate log segments
+ long maxCompletedLogSegmentSequenceNumber = -1L;
+ LogSegmentMetadata inprogressSegment = null;
+ for (LogSegmentMetadata segment : segments) {
+ if (!segment.isInProgress()) {
+ maxCompletedLogSegmentSequenceNumber = Math.max(maxCompletedLogSegmentSequenceNumber, segment.getLogSegmentSequenceNumber());
+ } else {
+ // we already found an inprogress segment
+ if (null != inprogressSegment) {
+ throw new DLIllegalStateException("Multiple inprogress segments found for stream " + streamName + " : " + segments);
+ }
+ inprogressSegment = segment;
+ }
+ }
+ if (null == inprogressSegment || inprogressSegment.getLogSegmentSequenceNumber() > maxCompletedLogSegmentSequenceNumber) {
+ // nothing to fix
+ return;
+ }
+ final long newLogSegmentSequenceNumber = maxCompletedLogSegmentSequenceNumber + 1;
+ if (interactive && !IOUtils.confirmPrompt("Confirm to fix (Y/N), Ctrl+C to break : ")) {
+ return;
+ }
+ final LogSegmentMetadata newSegment =
+ FutureUtils.result(metadataUpdater.changeSequenceNumber(inprogressSegment, newLogSegmentSequenceNumber));
+ LOG.info("Fixed {} : {} -> {} ",
+ new Object[] { streamName, inprogressSegment, newSegment });
+ if (verbose) {
+ System.out.println("Fixed " + streamName + " : " + inprogressSegment.getZNodeName()
+ + " -> " + newSegment.getZNodeName());
+ System.out.println("\t old: " + inprogressSegment);
+ System.out.println("\t new: " + newSegment);
+ System.out.println();
+ }
+ } finally {
+ dlm.close();
+ }
+ }
+
+ private static class LogSegmentCandidate {
+ final LogSegmentMetadata metadata;
+ final LogRecordWithDLSN lastRecord;
+
+ LogSegmentCandidate(LogSegmentMetadata metadata, LogRecordWithDLSN lastRecord) {
+ this.metadata = metadata;
+ this.lastRecord = lastRecord;
+ }
+
+ @Override
+ public String toString() {
+ return "LogSegmentCandidate[ metadata = " + metadata + ", last record = " + lastRecord + " ]";
+ }
+
+ }
+
+ private static final Comparator<LogSegmentCandidate> LOG_SEGMENT_CANDIDATE_COMPARATOR =
+ new Comparator<LogSegmentCandidate>() {
+ @Override
+ public int compare(LogSegmentCandidate o1, LogSegmentCandidate o2) {
+ return LogSegmentMetadata.COMPARATOR.compare(o1.metadata, o2.metadata);
+ }
+ };
+
+ private static class StreamCandidate {
+
+ final String streamName;
+ final SortedSet<LogSegmentCandidate> segmentCandidates =
+ new TreeSet<LogSegmentCandidate>(LOG_SEGMENT_CANDIDATE_COMPARATOR);
+
+ StreamCandidate(String streamName) {
+ this.streamName = streamName;
+ }
+
+ synchronized void addLogSegmentCandidate(LogSegmentCandidate segmentCandidate) {
+ segmentCandidates.add(segmentCandidate);
+ }
+
+ @Override
+ public String toString() {
+ return "StreamCandidate[ name = " + streamName + ", segments = " + segmentCandidates + " ]";
+ }
+ }
+
+ public static void checkAndRepairDLNamespace(final URI uri,
+ final DistributedLogNamespace namespace,
+ final MetadataUpdater metadataUpdater,
+ final OrderedScheduler scheduler,
+ final boolean verbose,
+ final boolean interactive) throws IOException {
+ checkAndRepairDLNamespace(uri, namespace, metadataUpdater, scheduler, verbose, interactive, 1);
+ }
+
+ public static void checkAndRepairDLNamespace(final URI uri,
+ final DistributedLogNamespace namespace,
+ final MetadataUpdater metadataUpdater,
+ final OrderedScheduler scheduler,
+ 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.
+ 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 under " + uri);
+ }
+ if (streams.size() == 0) {
+ System.out.println("+ 0. nothing to check. quit.");
+ return;
+ }
+ Map<String, StreamCandidate> streamCandidates =
+ checkStreams(namespace, streams, scheduler, concurrency);
+ if (verbose) {
+ System.out.println("+ 0. " + streamCandidates.size() + " corrupted streams found.");
+ }
+ if (interactive && !IOUtils.confirmPrompt("Do you want to fix all " + streamCandidates.size() + " corrupted streams (Y/N) : ")) {
+ return;
+ }
+ if (verbose) {
+ System.out.println("- 1. repairing " + streamCandidates.size() + " corrupted streams.");
+ }
+ for (StreamCandidate candidate : streamCandidates.values()) {
+ if (!repairStream(metadataUpdater, candidate, verbose, interactive)) {
+ if (verbose) {
+ System.out.println("* 1. aborted repairing corrupted streams.");
+ }
+ return;
+ }
+ }
+ if (verbose) {
+ System.out.println("+ 1. repaired " + streamCandidates.size() + " corrupted streams.");
+ }
+ }
+
+ private static Map<String, StreamCandidate> checkStreams(
+ final DistributedLogNamespace namespace,
+ final Collection<String> streams,
+ final OrderedScheduler scheduler,
+ final int concurrency) throws IOException {
+ final LinkedBlockingQueue<String> streamQueue =
+ new LinkedBlockingQueue<String>();
+ streamQueue.addAll(streams);
+ final Map<String, StreamCandidate> candidateMap =
+ new ConcurrentSkipListMap<String, StreamCandidate>();
+ final AtomicInteger numPendingStreams = new AtomicInteger(streams.size());
+ final CountDownLatch doneLatch = new CountDownLatch(1);
+ Runnable checkRunnable = new Runnable() {
+ @Override
+ public void run() {
+ while (!streamQueue.isEmpty()) {
+ String stream;
+ try {
+ stream = streamQueue.take();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ break;
+ }
+ StreamCandidate candidate;
+ try {
+ LOG.info("Checking stream {}.", stream);
+ candidate = checkStream(namespace, stream, scheduler);
+ LOG.info("Checked stream {} - {}.", stream, candidate);
+ } catch (IOException e) {
+ LOG.error("Error on checking stream {} : ", stream, e);
+ doneLatch.countDown();
+ break;
+ }
+ if (null != candidate) {
+ candidateMap.put(stream, candidate);
+ }
+ if (numPendingStreams.decrementAndGet() == 0) {
+ doneLatch.countDown();
+ }
+ }
+ }
+ };
+ Thread[] threads = new Thread[concurrency];
+ for (int i = 0; i < concurrency; i++) {
+ threads[i] = new Thread(checkRunnable, "check-thread-" + i);
+ threads[i].start();
+ }
+ try {
+ doneLatch.await();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ if (numPendingStreams.get() != 0) {
+ throw new IOException(numPendingStreams.get() + " streams left w/o checked");
+ }
+ for (int i = 0; i < concurrency; i++) {
+ threads[i].interrupt();
+ try {
+ threads[i].join();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+ return candidateMap;
+ }
+
+ private static StreamCandidate checkStream(
+ final DistributedLogNamespace namespace,
+ final String streamName,
+ final OrderedScheduler scheduler) throws IOException {
+ DistributedLogManager dlm = namespace.openLog(streamName);
+ try {
+ List<LogSegmentMetadata> segments = dlm.getLogSegments();
+ if (segments.isEmpty()) {
+ return null;
+ }
+ List<Future<LogSegmentCandidate>> futures =
+ new ArrayList<Future<LogSegmentCandidate>>(segments.size());
+ for (LogSegmentMetadata segment : segments) {
+ futures.add(checkLogSegment(namespace, streamName, segment, scheduler));
+ }
+ List<LogSegmentCandidate> segmentCandidates;
+ try {
+ segmentCandidates = Await.result(Future.collect(futures));
+ } catch (Exception e) {
+ throw new IOException("Failed on checking stream " + streamName, e);
+ }
+ StreamCandidate streamCandidate = new StreamCandidate(streamName);
+ for (LogSegmentCandidate segmentCandidate: segmentCandidates) {
+ if (null != segmentCandidate) {
+ streamCandidate.addLogSegmentCandidate(segmentCandidate);
+ }
+ }
+ if (streamCandidate.segmentCandidates.isEmpty()) {
+ return null;
+ }
+ return streamCandidate;
+ } finally {
+ dlm.close();
+ }
+ }
+
+ private static Future<LogSegmentCandidate> checkLogSegment(
+ final DistributedLogNamespace namespace,
+ final String streamName,
+ final LogSegmentMetadata metadata,
+ final OrderedScheduler scheduler) {
+ if (metadata.isInProgress()) {
+ return Future.value(null);
+ }
+
+ final LogSegmentEntryStore entryStore = namespace.getNamespaceDriver()
+ .getLogSegmentEntryStore(NamespaceDriver.Role.READER);
+ return ReadUtils.asyncReadLastRecord(
+ streamName,
+ metadata,
+ true,
+ false,
+ true,
+ 4,
+ 16,
+ new AtomicInteger(0),
+ scheduler,
+ entryStore
+ ).map(new Function<LogRecordWithDLSN, LogSegmentCandidate>() {
+ @Override
+ public LogSegmentCandidate apply(LogRecordWithDLSN record) {
+ if (null != record &&
+ (record.getDlsn().compareTo(metadata.getLastDLSN()) > 0 ||
+ record.getTransactionId() > metadata.getLastTxId() ||
+ !metadata.isRecordPositionWithinSegmentScope(record))) {
+ return new LogSegmentCandidate(metadata, record);
+ } else {
+ return null;
+ }
+ }
+ });
+ }
+
+ private static boolean repairStream(MetadataUpdater metadataUpdater,
+ StreamCandidate streamCandidate,
+ boolean verbose,
+ boolean interactive) throws IOException {
+ if (verbose) {
+ System.out.println("Stream " + streamCandidate.streamName + " : ");
+ for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
+ System.out.println(" " + segmentCandidate.metadata.getLogSegmentSequenceNumber()
+ + " : metadata = " + segmentCandidate.metadata + ", last dlsn = "
+ + segmentCandidate.lastRecord.getDlsn());
+ }
+ System.out.println("-------------------------------------------");
+ }
+ if (interactive && !IOUtils.confirmPrompt("Do you want to fix the stream " + streamCandidate.streamName + " (Y/N) : ")) {
+ return false;
+ }
+ for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
+ LogSegmentMetadata newMetadata = FutureUtils.result(
+ metadataUpdater.updateLastRecord(segmentCandidate.metadata, segmentCandidate.lastRecord));
+ if (verbose) {
+ System.out.println(" Fixed segment " + segmentCandidate.metadata.getLogSegmentSequenceNumber() + " : ");
+ System.out.println(" old metadata : " + segmentCandidate.metadata);
+ System.out.println(" new metadata : " + newMetadata);
+ }
+ }
+ if (verbose) {
+ System.out.println("-------------------------------------------");
+ }
+ return true;
+ }
+
+ //
+ // Commands
+ //
+
+ /**
+ * Unbind the bookkeeper environment for a given distributedlog uri.
+ *
+ * TODO: move unbind operation to namespace driver
+ */
+ class UnbindCommand extends OptsCommand {
+
+ Options options = new Options();
+
+ UnbindCommand() {
+ super("unbind", "unbind the bookkeeper environment bound for a given distributedlog instance.");
+ options.addOption("f", "force", false, "Force unbinding without prompt.");
+ }
+
+ @Override
+ protected Options getOptions() {
+ return options;
+ }
+
+ @Override
+ protected String getUsage() {
+ return "unbind [options] <distributedlog uri>";
+ }
+
+ @Override
+ protected int runCmd(CommandLine cmdline) throws Exception {
+ String[] args = cmdline.getArgs();
+ if (args.length <= 0) {
+ System.err.println("No distributedlog uri specified.");
+ printUsage();
+ return -1;
+ }
+ boolean force = cmdline.hasOption("f");
+ URI uri = URI.create(args[0]);
+ // resolving the uri to see if there is another bindings in this uri.
+ ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri)
+ .sessionTimeoutMs(10000).build();
+ BKDLConfig bkdlConfig;
+ try {
+ bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+ } catch (IOException ie) {
+ bkdlConfig = null;
+ }
+ if (null == bkdlConfig) {
+ System.out.println("No bookkeeper is bound to " + uri);
+ return 0;
+ } else {
+ System.out.println("There is bookkeeper bound to " + uri + " : ");
+ System.out.println("");
+ System.out.println(bkdlConfig.toString());
+ System.out.println("");
+ if (!force && !IOUtils.confirmPrompt("Do you want to unbind " + uri + " :\n")) {
+ return 0;
+ }
+ }
+ DLMetadata.unbind(uri);
+ System.out.println("Unbound on " + uri + ".");
+ return 0;
+ }
+ }
+
+ /**
+ * Bind Command to bind bookkeeper environment for a given distributed uri.
+ *
+ * TODO: move bind to namespace driver
+ */
+ class BindCommand extends OptsCommand {
+
+ Options options = new Options();
+
+ BindCommand() {
+ super("bind", "bind the bookkeeper environment settings for a given distributedlog instance.");
+ options.addOption("l", "bkLedgers", true, "ZooKeeper ledgers path for bookkeeper instance.");
+ options.addOption("s", "bkZkServers", true, "ZooKeeper servers used for bookkeeper for writers.");
+ options.addOption("bkzr", "bkZkServersForReader", true, "ZooKeeper servers used for bookkeeper for readers.");
+ options.addOption("dlzw", "dlZkServersForWriter", true, "ZooKeeper servers used for distributedlog for writers.");
+ options.addOption("dlzr", "dlZkServersForReader", true, "ZooKeeper servers used for distributedlog for readers.");
+ options.addOption("i", "sanityCheckTxnID", true, "Flag to sanity check highest txn id.");
+ options.addOption("r", "encodeRegionID", true, "Flag to encode region id.");
+ options.addOption("seqno", "firstLogSegmentSeqNo", true, "The first log segment sequence number to use after upgrade");
+ options.addOption("fns", "federatedNamespace", false, "Flag to turn a namespace to federated namespace");
+ options.addOption("f", "force", false, "Force binding without prompt.");
+ options.addOption("c", "creation", false, "Whether is it a creation binding.");
+ options.addOption("q", "query", false, "Query the bookkeeper bindings");
+ }
+
+ @Override
+ protected Options getOptions() {
+ return options;
+ }
+
+ @Override
+ protected String getUsage() {
+ return "bind [options] <distributedlog uri>";
+ }
+
+ @Override
+ protected int runCmd(CommandLine cmdline) throws Exception {
+ boolean isQuery = cmdline.hasOption("q");
+ if (!isQuery && (!cmdline.hasOption("l") || !cmdline.hasOption("s"))) {
+ System.err.println("Error: Neither zkServers nor ledgersPath specified for bookkeeper environment.");
+ printUsage();
+ return -1;
+ }
+ String[] args = cmdline.getArgs();
+ if (args.length <= 0) {
+ System.err.println("No distributedlog uri specified.");
+ printUsage();
+ return -1;
+ }
+ boolean force = cmdline.hasOption("f");
+ boolean creation = cmdline.hasOption("c");
+ String bkLedgersPath = cmdline.getOptionValue("l");
+ String bkZkServersForWriter = cmdline.getOptionValue("s");
+ boolean sanityCheckTxnID =
+ !cmdline.hasOption("i") || Boolean.parseBoolean(cmdline.getOptionValue("i"));
+ boolean encodeRegionID =
+ cmdline.hasOption("r") && Boolean.parseBoolean(cmdline.getOptionValue("r"));
+
+ String bkZkServersForReader;
+ if (cmdline.hasOption("bkzr")) {
+ bkZkServersForReader = cmdline.getOptionValue("bkzr");
+ } else {
+ bkZkServersForReader = bkZkServersForWriter;
+ }
+
+ URI uri = URI.create(args[0]);
+
+ String dlZkServersForWriter;
+ String dlZkServersForReader;
+ if (cmdline.hasOption("dlzw")) {
+ dlZkServersForWriter = cmdline.getOptionValue("dlzw");
+ } else {
+ dlZkServersForWriter = BKNamespaceDriver.getZKServersFromDLUri(uri);
+ }
+ if (cmdline.hasOption("dlzr")) {
+ dlZkServersForReader = cmdline.getOptionValue("dlzr");
+ } else {
+ dlZkServersForReader = dlZkServersForWriter;
+ }
+
+ // resolving the uri to see if there is another bindings in this uri.
+ ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri).zkAclId(null)
+ .sessionTimeoutMs(10000).build();
+ try {
+ BKDLConfig newBKDLConfig =
+ new BKDLConfig(dlZkServersForWriter, dlZkServersForReader,
+ bkZkServersForWriter, bkZkServersForReader, bkLedgersPath)
+ .setSanityCheckTxnID(sanityCheckTxnID)
+ .setEncodeRegionID(encodeRegionID);
+
+ if (cmdline.hasOption("seqno")) {
+ newBKDLConfig = newBKDLConfig.setFirstLogSegmentSeqNo(Long.parseLong(cmdline.getOptionValue("seqno")));
+ }
+
+ if (cmdline.hasOption("fns")) {
+ newBKDLConfig = newBKDLConfig.setFederatedNamespace(true);
+ }
+
+ BKDLConfig bkdlConfig;
+ try {
+ bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+ } catch (IOException ie) {
+ bkdlConfig = null;
+ }
+ if (null == bkdlConfig) {
+ System.out.println("No bookkeeper is bound to " + uri);
+ } else {
+ System.out.println("There is bookkeeper bound to " + uri + " : ");
+ System.out.println("");
+ System.out.println(bkdlConfig.toString());
+ System.out.println("");
+ if (!isQuery) {
+ if (newBKDLConfig.equals(bkdlConfig)) {
+ System.out.println("No bookkeeper binding needs to be updated. Quit.");
+ return 0;
+ } else if(!newBKDLConfig.isFederatedNamespace() && bkdlConfig.isFederatedNamespace()) {
+ System.out.println("You can't turn a federated namespace back to non-federated.");
+ return 0;
+ } else {
+ if (!force && !IOUtils.confirmPrompt("Do you want to bind " + uri
+ + " with new bookkeeper instance :\n" + newBKDLConfig)) {
+ return 0;
+ }
+ }
+ }
+ }
+ if (isQuery) {
+ System.out.println("Done.");
+ return 0;
+ }
+ DLMetadata dlMetadata = DLMetadata.create(newBKDLConfig);
+ if (creation) {
+ try {
+ dlMetadata.create(uri);
+ System.out.println("Created binding on " + uri + ".");
+ } catch (IOException ie) {
+ System.err.println("Failed to create binding on " + uri + " : " + ie.getMessage());
+ }
+ } else {
+ try {
+ dlMetadata.update(uri);
+ System.out.println("Updated binding on " + uri + " : ");
+ System.out.println("");
+ System.out.println(newBKDLConfig.toString());
+ System.out.println("");
+ } catch (IOException ie) {
+ System.err.println("Failed to update binding on " + uri + " : " + ie.getMessage());
+ }
+ }
+ if (newBKDLConfig.isFederatedNamespace()) {
+ try {
+ FederatedZKLogMetadataStore.createFederatedNamespace(uri, zkc);
+ } catch (KeeperException.NodeExistsException nee) {
+ // ignore node exists exception
+ }
+ }
+ return 0;
+ } finally {
+ zkc.close();
+ }
+ }
+ }
+
+ static class RepairSeqNoCommand extends PerDLCommand {
+
+ boolean dryrun = false;
+ boolean verbose = false;
+ final List<String> streams = new ArrayList<String>();
+
+ RepairSeqNoCommand() {
+ super("repairseqno", "Repair a stream whose inprogress log segment has lower sequence number.");
+ options.addOption("d", "dryrun", false, "Dry run without repairing");
+ options.addOption("l", "list", true, "List of streams to repair, separated by comma");
+ options.addOption("v", "verbose", false, "Print verbose messages");
+ }
+
+ @Override
+ protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+ super.parseCommandLine(cmdline);
+ dryrun = cmdline.hasOption("d");
+ verbose = cmdline.hasOption("v");
+ force = !dryrun && cmdline.hasOption("f");
+ if (!cmdline.hasOption("l")) {
+ throw new ParseException("No streams provided to repair");
+ }
+ String streamsList = cmdline.getOptionValue("l");
+ Collections.addAll(streams, streamsList.split(","));
+ }
+
+ @Override
+ protected int runCmd() throws Exception {
+ MetadataUpdater metadataUpdater = dryrun ?
+ new DryrunLogSegmentMetadataStoreUpdater(getConf(),
+ getLogSegmentMetadataStore()) :
+ LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
+ getLogSegmentMetadataStore());
+ System.out.println("List of streams : ");
+ System.out.println(streams);
+ if (!IOUtils.confirmPrompt("Do you want to repair all these streams (Y/N):")) {
+ return -1;
+ }
+ for (String stream : streams) {
+ fixInprogressSegmentWithLowerSequenceNumber(getNamespace(), metadataUpdater, stream, verbose, !getForce());
+ }
+ return 0;
+ }
+
+ @Override
+ protected String getUsage() {
+ return "repairseqno [options]";
+ }
+ }
+
+ static class DLCKCommand extends PerDLCommand {
+
+ boolean dryrun = false;
+ boolean verbose = false;
+ int concurrency = 1;
+
+ DLCKCommand() {
+ super("dlck", "Check and repair a distributedlog namespace");
+ options.addOption("d", "dryrun", false, "Dry run without repairing");
+ options.addOption("v", "verbose", false, "Print verbose messages");
+ options.addOption("cy", "concurrency", true, "Concurrency on checking streams");
+ }
+
+ @Override
+ protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+ super.parseCommandLine(cmdline);
+ dryrun = cmdline.hasOption("d");
+ verbose = cmdline.hasOption("v");
+ if (cmdline.hasOption("cy")) {
+ try {
+ concurrency = Integer.parseInt(cmdline.getOptionValue("cy"));
+ } catch (NumberFormatException nfe) {
+ throw new ParseException("Invalid concurrency value : " + cmdline.getOptionValue("cy"));
+ }
+ }
+ }
+
+ @Override
+ protected int runCmd() throws Exception {
+ MetadataUpdater metadataUpdater = dryrun ?
+ new DryrunLogSegmentMetadataStoreUpdater(getConf(),
+ getLogSegmentMetadataStore()) :
+ LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
+ getLogSegmentMetadataStore());
+ OrderedScheduler scheduler = OrderedScheduler.newBuilder()
+ .name("dlck-scheduler")
+ .corePoolSize(Runtime.getRuntime().availableProcessors())
+ .build();
+ ExecutorService executorService = Executors.newCachedThreadPool();
+ try {
+ checkAndRepairDLNamespace(getUri(), getNamespace(), metadataUpdater, scheduler,
+ verbose, !getForce(), concurrency);
+ } finally {
+ SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
+ }
+ return 0;
+ }
+
+ @Override
+ protected String getUsage() {
+ return "dlck [options]";
+ }
+ }
+
+ static class DeleteStreamACLCommand extends PerDLCommand {
+
+ String stream = null;
+
+ DeleteStreamACLCommand() {
+ super("delete_stream_acl", "Delete ACL for a given stream");
+ options.addOption("s", "stream", true, "Stream to set ACL");
+ }
+
+ @Override
+ protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+ super.parseCommandLine(cmdline);
+ if (!cmdline.hasOption("s")) {
+ throw new ParseException("No stream to set ACL");
+ }
+ stream = cmdline.getOptionValue("s");
+ }
+
+ @Override
+ protected int runCmd() throws Exception {
+ BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(getZooKeeperClient(), getUri());
+ if (null == bkdlConfig.getACLRootPath()) {
+ // acl isn't enabled for this namespace.
+ System.err.println("ACL isn't enabled for namespace " + getUri());
+ return -1;
+ }
+ String zkPath = getUri() + "/" + bkdlConfig.getACLRootPath() + "/" + stream;
+ ZKAccessControl.delete(getZooKeeperClient(), zkPath);
+ return 0;
+ }
+
+ @Override
+ protected String getUsage() {
+ return null;
+ }
+ }
+
+ static class SetStreamACLCommand extends SetACLCommand {
+
+ String stream = null;
+
+ SetStreamACLCommand() {
+ super("set_stream_acl", "Set Default ACL for a given stream");
+ options.addOption("s", "stream", true, "Stream to set ACL");
+ }
+
+ @Override
+ protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+ super.parseCommandLine(cmdline);
+ if (!cmdline.hasOption("s")) {
+ throw new ParseException("No stream to set ACL");
+ }
+ stream = cmdline.getOptionValue("s");
+ }
+
+ @Override
+ protected String getZKPath(String zkRootPath) {
+ return zkRootPath + "/" + stream;
+ }
+
+ @Override
+ protected String getUsage() {
+ return "set_stream_acl [options]";
+ }
+ }
+
+ static class SetDefaultACLCommand extends SetACLCommand {
+
+ SetDefaultACLCommand() {
+ super("set_default_acl", "Set Default ACL for a namespace");
+ }
+
+ @Override
+ protected String getZKPath(String zkRootPath) {
+ return zkRootPath;
+ }
+
+ @Override
+ protected String getUsage() {
+ return "set_default_acl [options]";
+ }
+ }
+
+ static abstract class SetACLCommand extends PerDLCommand {
+
+ boolean denyWrite = false;
+ boolean denyTruncate = false;
+ boolean denyDelete = false;
+ boolean denyAcquire = false;
+ boolean denyRelease = false;
+
+ protected SetACLCommand(String name, String description) {
+ super(name, description);
+ options.addOption("dw", "deny-write", false, "Deny write/bulkWrite requests");
+ options.addOption("dt", "deny-truncate", false, "Deny truncate requests");
+ options.addOption("dd", "deny-delete", false, "Deny delete requests");
+ options.addOption("da", "deny-acquire", false, "Deny acquire requests");
+ options.addOption("dr", "deny-release", false, "Deny release requests");
+ }
+
+ @Override
+ protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+ super.parseCommandLine(cmdline);
+ denyWrite = cmdline.hasOption("dw");
+ denyTruncate = cmdline.hasOption("dt");
+ denyDelete = cmdline.hasOption("dd");
+ denyAcquire = cmdline.hasOption("da");
+ denyRelease = cmdline.hasOption("dr");
+ }
+
+ protected abstract String getZKPath(String zkRootPath);
+
+ protected ZKAccessControl getZKAccessControl(ZooKeeperClient zkc, String zkPath) throws Exception {
+ ZKAccessControl accessControl;
+ try {
+ accessControl = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+ } catch (KeeperException.NoNodeException nne) {
+ accessControl = new ZKAccessControl(new AccessControlEntry(), zkPath);
+ }
+ return accessControl;
+ }
+
+ protected void setZKAccessControl(ZooKeeperClient zkc, ZKAccessControl accessControl) throws Exception {
+ String zkPath = accessControl.getZKPath();
+ if (null == zkc.get().exists(zkPath, false)) {
+ accessControl.create(zkc);
+ } else {
+ accessControl.update(zkc);
+ }
+ }
+
+ @Override
+ protected int runCmd() throws Exception {
+ BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(getZooKeeperClient(), getUri());
+ if (null == bkdlConfig.getACLRootPath()) {
+ // acl isn't enabled for this namespace.
+ System.err.println("ACL isn't enabled for namespace " + getUri());
+ return -1;
+ }
+ String zkPath = getZKPath(getUri().getPath() + "/" + bkdlConfig.getACLRootPath());
+ ZKAccessControl accessControl = getZKAccessControl(getZooKeeperClient(), zkPath);
+ AccessControlEntry acl = accessControl.getAccessControlEntry();
+ acl.setDenyWrite(denyWrite);
+ acl.setDenyTruncate(denyTruncate);
+ acl.setDenyDelete(denyDelete);
+ acl.setDenyAcquire(denyAcquire);
+ acl.setDenyRelease(denyRelease);
+ setZKAccessControl(getZooKeeperClient(), accessControl);
+ return 0;
+ }
+
+ }
+
+ public DistributedLogAdmin() {
+ super();
+ commands.clear();
+ addCommand(new HelpCommand());
+ addCommand(new BindCommand());
+ addCommand(new UnbindCommand());
+ addCommand(new RepairSeqNoCommand());
+ addCommand(new DLCKCommand());
+ addCommand(new SetDefaultACLCommand());
+ addCommand(new SetStreamACLCommand());
+ addCommand(new DeleteStreamACLCommand());
+ }
+
+ @Override
+ protected String getName() {
+ return "dlog_admin";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java
new file mode 100644
index 0000000..d708111
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Admin Tools for DistributedLog
+ */
+package org.apache.distributedlog.admin;
[50/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Benchmarker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Benchmarker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Benchmarker.java
deleted file mode 100644
index 5b04a05..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Benchmarker.java
+++ /dev/null
@@ -1,468 +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.benchmark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.benchmark.utils.ShiftableRateLimiter;
-import com.twitter.finagle.stats.OstrichStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The launcher for benchmarks.
- */
-public class Benchmarker {
-
- private static final Logger logger = LoggerFactory.getLogger(Benchmarker.class);
-
- static final String USAGE = "Benchmarker [-u <uri>] [-c <conf>] [-s serverset] [-m (read|write|dlwrite)]";
-
- final String[] args;
- final Options options = new Options();
-
- int rate = 100;
- int maxRate = 1000;
- int changeRate = 100;
- int changeRateSeconds = 1800;
- int concurrency = 10;
- String streamPrefix = "dlog-loadtest";
- int shardId = -1;
- int numStreams = 10;
- List<String> serversetPaths = new ArrayList<String>();
- List<String> finagleNames = new ArrayList<String>();
- int msgSize = 256;
- String mode = null;
- int durationMins = 60;
- URI dlUri = null;
- int batchSize = 0;
- int readersPerStream = 1;
- Integer maxStreamId = null;
- int truncationInterval = 3600;
- Integer startStreamId = null;
- Integer endStreamId = null;
- int hostConnectionCoreSize = 10;
- int hostConnectionLimit = 10;
- boolean thriftmux = false;
- boolean handshakeWithClientInfo = false;
- boolean readFromHead = false;
- int sendBufferSize = 1024 * 1024;
- int recvBufferSize = 1024 * 1024;
- boolean enableBatching = false;
- int batchBufferSize = 256 * 1024;
- int batchFlushIntervalMicros = 2000;
- String routingServiceFinagleNameString;
-
- final DistributedLogConfiguration conf = new DistributedLogConfiguration();
- final StatsReceiver statsReceiver = new OstrichStatsReceiver();
- StatsProvider statsProvider = null;
-
- Benchmarker(String[] args) {
- this.args = args;
- // prepare options
- options.addOption("s", "serverset", true, "Proxy Server Set (separated by ',')");
- options.addOption("fn", "finagle-name", true, "Write proxy finagle name (separated by ',')");
- options.addOption("c", "conf", true, "DistributedLog Configuration File");
- options.addOption("u", "uri", true, "DistributedLog URI");
- options.addOption("i", "shard", true, "Shard Id");
- options.addOption("p", "provider", true, "DistributedLog Stats Provider");
- options.addOption("d", "duration", true, "Duration (minutes)");
- options.addOption("sp", "streamprefix", true, "Stream Prefix");
- options.addOption("sc", "streamcount", true, "Number of Streams");
- options.addOption("ms", "messagesize", true, "Message Size (bytes)");
- options.addOption("bs", "batchsize", true, "Batch Size");
- options.addOption("r", "rate", true, "Rate limit (requests/second)");
- options.addOption("mr", "max-rate", true, "Maximum Rate limit (requests/second)");
- options.addOption("cr", "change-rate", true, "Rate to increase each change period (requests/second)");
- options.addOption("ci", "change-interval", true, "Rate to increase period, seconds");
- options.addOption("t", "concurrency", true, "Concurrency (number of threads)");
- options.addOption("m", "mode", true, "Benchmark mode (read/write)");
- options.addOption("rps", "readers-per-stream", true, "Number readers per stream");
- options.addOption("msid", "max-stream-id", true, "Max Stream ID");
- options.addOption("ti", "truncation-interval", true, "Truncation interval in seconds");
- options.addOption("ssid", "start-stream-id", true, "Start Stream ID");
- options.addOption("esid", "end-stream-id", true, "Start Stream ID");
- options.addOption("hccs", "host-connection-core-size", true, "Finagle hostConnectionCoreSize");
- options.addOption("hcl", "host-connection-limit", true, "Finagle hostConnectionLimit");
- options.addOption("mx", "thriftmux", false, "Enable thriftmux (write mode only)");
- options.addOption("hsci", "handshake-with-client-info", false, "Enable handshaking with client info");
- options.addOption("rfh", "read-from-head", false, "Read from head of the stream");
- options.addOption("sb", "send-buffer", true, "Channel send buffer size, in bytes");
- options.addOption("rb", "recv-buffer", true, "Channel recv buffer size, in bytes");
- options.addOption("bt", "enable-batch", false, "Enable batching on writers");
- options.addOption("bbs", "batch-buffer-size", true, "The batch buffer size in bytes");
- options.addOption("bfi", "batch-flush-interval", true, "The batch buffer flush interval in micros");
- options.addOption("rs", "routing-service", true, "The routing service finagle name for server-side routing");
- options.addOption("h", "help", false, "Print usage.");
- }
-
- void printUsage() {
- HelpFormatter helpFormatter = new HelpFormatter();
- helpFormatter.printHelp(USAGE, options);
- }
-
- void run() throws Exception {
- logger.info("Running benchmark.");
-
- BasicParser parser = new BasicParser();
- CommandLine cmdline = parser.parse(options, args);
- if (cmdline.hasOption("h")) {
- printUsage();
- System.exit(0);
- }
- if (cmdline.hasOption("s")) {
- String serversetPathStr = cmdline.getOptionValue("s");
- serversetPaths = Arrays.asList(StringUtils.split(serversetPathStr, ','));
- }
- if (cmdline.hasOption("fn")) {
- String finagleNameStr = cmdline.getOptionValue("fn");
- finagleNames = Arrays.asList(StringUtils.split(finagleNameStr, ','));
- }
- if (cmdline.hasOption("i")) {
- shardId = Integer.parseInt(cmdline.getOptionValue("i"));
- }
- if (cmdline.hasOption("d")) {
- durationMins = Integer.parseInt(cmdline.getOptionValue("d"));
- }
- if (cmdline.hasOption("sp")) {
- streamPrefix = cmdline.getOptionValue("sp");
- }
- if (cmdline.hasOption("sc")) {
- numStreams = Integer.parseInt(cmdline.getOptionValue("sc"));
- }
- if (cmdline.hasOption("ms")) {
- msgSize = Integer.parseInt(cmdline.getOptionValue("ms"));
- }
- if (cmdline.hasOption("r")) {
- rate = Integer.parseInt(cmdline.getOptionValue("r"));
- }
- if (cmdline.hasOption("mr")) {
- maxRate = Integer.parseInt(cmdline.getOptionValue("mr"));
- }
- if (cmdline.hasOption("cr")) {
- changeRate = Integer.parseInt(cmdline.getOptionValue("cr"));
- }
- if (cmdline.hasOption("ci")) {
- changeRateSeconds = Integer.parseInt(cmdline.getOptionValue("ci"));
- }
- if (cmdline.hasOption("t")) {
- concurrency = Integer.parseInt(cmdline.getOptionValue("t"));
- }
- if (cmdline.hasOption("m")) {
- mode = cmdline.getOptionValue("m");
- }
- if (cmdline.hasOption("u")) {
- dlUri = URI.create(cmdline.getOptionValue("u"));
- }
- if (cmdline.hasOption("bs")) {
- batchSize = Integer.parseInt(cmdline.getOptionValue("bs"));
- checkArgument("write" != mode, "batchSize supported only for mode=write");
- }
- if (cmdline.hasOption("c")) {
- String configFile = cmdline.getOptionValue("c");
- conf.loadConf(new File(configFile).toURI().toURL());
- }
- if (cmdline.hasOption("rps")) {
- readersPerStream = Integer.parseInt(cmdline.getOptionValue("rps"));
- }
- if (cmdline.hasOption("msid")) {
- maxStreamId = Integer.parseInt(cmdline.getOptionValue("msid"));
- }
- if (cmdline.hasOption("ti")) {
- truncationInterval = Integer.parseInt(cmdline.getOptionValue("ti"));
- }
- if (cmdline.hasOption("ssid")) {
- startStreamId = Integer.parseInt(cmdline.getOptionValue("ssid"));
- }
- if (cmdline.hasOption("esid")) {
- endStreamId = Integer.parseInt(cmdline.getOptionValue("esid"));
- }
- if (cmdline.hasOption("hccs")) {
- hostConnectionCoreSize = Integer.parseInt(cmdline.getOptionValue("hccs"));
- }
- if (cmdline.hasOption("hcl")) {
- hostConnectionLimit = Integer.parseInt(cmdline.getOptionValue("hcl"));
- }
- if (cmdline.hasOption("sb")) {
- sendBufferSize = Integer.parseInt(cmdline.getOptionValue("sb"));
- }
- if (cmdline.hasOption("rb")) {
- recvBufferSize = Integer.parseInt(cmdline.getOptionValue("rb"));
- }
- if (cmdline.hasOption("rs")) {
- routingServiceFinagleNameString = cmdline.getOptionValue("rs");
- }
- thriftmux = cmdline.hasOption("mx");
- handshakeWithClientInfo = cmdline.hasOption("hsci");
- readFromHead = cmdline.hasOption("rfh");
- enableBatching = cmdline.hasOption("bt");
- if (cmdline.hasOption("bbs")) {
- batchBufferSize = Integer.parseInt(cmdline.getOptionValue("bbs"));
- }
- if (cmdline.hasOption("bfi")) {
- batchFlushIntervalMicros = Integer.parseInt(cmdline.getOptionValue("bfi"));
- }
-
- checkArgument(shardId >= 0, "shardId must be >= 0");
- checkArgument(numStreams > 0, "numStreams must be > 0");
- checkArgument(durationMins > 0, "durationMins must be > 0");
- checkArgument(streamPrefix != null, "streamPrefix must be defined");
- checkArgument(hostConnectionCoreSize > 0, "host connection core size must be > 0");
- checkArgument(hostConnectionLimit > 0, "host connection limit must be > 0");
-
- if (cmdline.hasOption("p")) {
- statsProvider = ReflectionUtils.newInstance(cmdline.getOptionValue("p"), StatsProvider.class);
- } else {
- statsProvider = new NullStatsProvider();
- }
-
- logger.info("Starting stats provider : {}.", statsProvider.getClass());
- statsProvider.start(conf);
-
- Worker w = null;
- if (mode.startsWith("read")) {
- w = runReader();
- } else if (mode.startsWith("write")) {
- w = runWriter();
- } else if (mode.startsWith("dlwrite")) {
- w = runDLWriter();
- } else if (mode.startsWith("dlread")) {
- w = runDLReader();
- }
-
- if (w == null) {
- throw new IOException("Unknown mode " + mode + " to run the benchmark.");
- }
-
- Thread workerThread = new Thread(w, mode + "-benchmark-thread");
- workerThread.start();
-
- TimeUnit.MINUTES.sleep(durationMins);
-
- logger.info("{} minutes passed, exiting...", durationMins);
- w.close();
-
- if (null != statsProvider) {
- statsProvider.stop();
- }
-
- Runtime.getRuntime().exit(0);
- }
-
- Worker runWriter() {
- checkArgument(!finagleNames.isEmpty() || !serversetPaths.isEmpty() || null != dlUri,
- "either serverset paths, finagle-names or uri required");
- checkArgument(msgSize > 0, "messagesize must be greater than 0");
- checkArgument(rate > 0, "rate must be greater than 0");
- checkArgument(maxRate >= rate, "max rate must be greater than rate");
- checkArgument(changeRate >= 0, "change rate must be positive");
- checkArgument(changeRateSeconds >= 0, "change rate must be positive");
- checkArgument(concurrency > 0, "concurrency must be greater than 0");
-
- ShiftableRateLimiter rateLimiter =
- new ShiftableRateLimiter(rate, maxRate, changeRate, changeRateSeconds, TimeUnit.SECONDS);
- return createWriteWorker(
- streamPrefix,
- dlUri,
- null == startStreamId ? shardId * numStreams : startStreamId,
- null == endStreamId ? (shardId + 1) * numStreams : endStreamId,
- rateLimiter,
- concurrency,
- msgSize,
- batchSize,
- hostConnectionCoreSize,
- hostConnectionLimit,
- serversetPaths,
- finagleNames,
- statsReceiver.scope("write_client"),
- statsProvider.getStatsLogger("write"),
- thriftmux,
- handshakeWithClientInfo,
- sendBufferSize,
- recvBufferSize,
- enableBatching,
- batchBufferSize,
- batchFlushIntervalMicros,
- routingServiceFinagleNameString);
- }
-
- protected WriterWorker createWriteWorker(
- String streamPrefix,
- URI uri,
- int startStreamId,
- int endStreamId,
- ShiftableRateLimiter rateLimiter,
- int writeConcurrency,
- int messageSizeBytes,
- int batchSize,
- int hostConnectionCoreSize,
- int hostConnectionLimit,
- List<String> serverSetPaths,
- List<String> finagleNames,
- StatsReceiver statsReceiver,
- StatsLogger statsLogger,
- boolean thriftmux,
- boolean handshakeWithClientInfo,
- int sendBufferSize,
- int recvBufferSize,
- boolean enableBatching,
- int batchBufferSize,
- int batchFlushIntervalMicros,
- String routingServiceFinagleNameString) {
- return new WriterWorker(
- streamPrefix,
- uri,
- startStreamId,
- endStreamId,
- rateLimiter,
- writeConcurrency,
- messageSizeBytes,
- batchSize,
- hostConnectionCoreSize,
- hostConnectionLimit,
- serverSetPaths,
- finagleNames,
- statsReceiver,
- statsLogger,
- thriftmux,
- handshakeWithClientInfo,
- sendBufferSize,
- recvBufferSize,
- enableBatching,
- batchBufferSize,
- batchFlushIntervalMicros,
- routingServiceFinagleNameString);
- }
-
- Worker runDLWriter() throws IOException {
- checkNotNull(dlUri, "dlUri must be defined");
- checkArgument(rate > 0, "rate must be greater than 0");
- checkArgument(maxRate >= rate, "max rate must be greater than rate");
- checkArgument(changeRate >= 0, "change rate must be positive");
- checkArgument(changeRateSeconds >= 0, "change rate must be positive");
- checkArgument(concurrency > 0, "concurrency must be greater than 0");
-
- ShiftableRateLimiter rateLimiter =
- new ShiftableRateLimiter(rate, maxRate, changeRate, changeRateSeconds, TimeUnit.SECONDS);
-
- return new DLWriterWorker(conf,
- dlUri,
- streamPrefix,
- shardId * numStreams,
- (shardId + 1) * numStreams,
- rateLimiter,
- concurrency,
- msgSize,
- statsProvider.getStatsLogger("dlwrite"));
- }
-
- Worker runReader() throws IOException {
- checkArgument(!finagleNames.isEmpty() || !serversetPaths.isEmpty() || null != dlUri,
- "either serverset paths, finagle-names or dlUri required");
- checkArgument(concurrency > 0, "concurrency must be greater than 0");
- checkArgument(truncationInterval > 0, "truncation interval should be greater than 0");
- return runReaderInternal(serversetPaths, finagleNames, truncationInterval);
- }
-
- Worker runDLReader() throws IOException {
- return runReaderInternal(new ArrayList<String>(), new ArrayList<String>(), 0);
- }
-
- private Worker runReaderInternal(List<String> serversetPaths,
- List<String> finagleNames,
- int truncationInterval) throws IOException {
- checkNotNull(dlUri);
-
- int ssid = null == startStreamId ? shardId * numStreams : startStreamId;
- int esid = null == endStreamId ? (shardId + readersPerStream) * numStreams : endStreamId;
- if (null != maxStreamId) {
- esid = Math.min(esid, maxStreamId);
- }
-
- return createReaderWorker(
- conf,
- dlUri,
- streamPrefix,
- ssid,
- esid,
- concurrency,
- serversetPaths,
- finagleNames,
- truncationInterval,
- readFromHead,
- statsReceiver,
- statsProvider.getStatsLogger("dlreader"));
- }
-
- protected ReaderWorker createReaderWorker(
- DistributedLogConfiguration conf,
- URI uri,
- String streamPrefix,
- int startStreamId,
- int endStreamId,
- int readThreadPoolSize,
- List<String> serverSetPaths,
- List<String> finagleNames,
- int truncationIntervalInSeconds,
- boolean readFromHead, /* read from the earliest data of log */
- StatsReceiver statsReceiver,
- StatsLogger statsLogger) throws IOException {
- return new ReaderWorker(
- conf,
- uri,
- streamPrefix,
- startStreamId,
- endStreamId,
- readThreadPoolSize,
- serverSetPaths,
- finagleNames,
- truncationIntervalInSeconds,
- readFromHead,
- statsReceiver,
- statsLogger);
- }
-
- public static void main(String[] args) {
- Benchmarker benchmarker = new Benchmarker(args);
- try {
- benchmarker.run();
- } catch (Exception e) {
- logger.info("Benchmark quit due to : ", e);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/DLWriterWorker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/DLWriterWorker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/DLWriterWorker.java
deleted file mode 100644
index 152cd32..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/DLWriterWorker.java
+++ /dev/null
@@ -1,245 +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.benchmark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.benchmark.utils.ShiftableRateLimiter;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.util.FutureEventListener;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The benchmark for core library writer.
- */
-public class DLWriterWorker implements Worker {
-
- private static final Logger LOG = LoggerFactory.getLogger(DLWriterWorker.class);
-
- static final int BACKOFF_MS = 200;
-
- final String streamPrefix;
- final int startStreamId;
- final int endStreamId;
- final int writeConcurrency;
- final int messageSizeBytes;
- final ExecutorService executorService;
- final ScheduledExecutorService rescueService;
- final ShiftableRateLimiter rateLimiter;
- final Random random;
- final DistributedLogNamespace namespace;
- final List<DistributedLogManager> dlms;
- final List<AsyncLogWriter> streamWriters;
- final int numStreams;
-
- volatile boolean running = true;
-
- final StatsLogger statsLogger;
- final OpStatsLogger requestStat;
-
- public DLWriterWorker(DistributedLogConfiguration conf,
- URI uri,
- String streamPrefix,
- int startStreamId,
- int endStreamId,
- ShiftableRateLimiter rateLimiter,
- int writeConcurrency,
- int messageSizeBytes,
- StatsLogger statsLogger) throws IOException {
- checkArgument(startStreamId <= endStreamId);
- this.streamPrefix = streamPrefix;
- this.startStreamId = startStreamId;
- this.endStreamId = endStreamId;
- this.rateLimiter = rateLimiter;
- this.writeConcurrency = writeConcurrency;
- this.messageSizeBytes = messageSizeBytes;
- this.statsLogger = statsLogger;
- this.requestStat = this.statsLogger.getOpStatsLogger("requests");
- this.executorService = Executors.newCachedThreadPool();
- this.rescueService = Executors.newSingleThreadScheduledExecutor();
- this.random = new Random(System.currentTimeMillis());
-
- this.namespace = DistributedLogNamespaceBuilder.newBuilder()
- .conf(conf)
- .uri(uri)
- .statsLogger(statsLogger.scope("dl"))
- .build();
- this.numStreams = endStreamId - startStreamId;
- dlms = new ArrayList<DistributedLogManager>(numStreams);
- streamWriters = new ArrayList<AsyncLogWriter>(numStreams);
- final ConcurrentMap<String, AsyncLogWriter> writers = new ConcurrentHashMap<String, AsyncLogWriter>();
- final CountDownLatch latch = new CountDownLatch(this.numStreams);
- for (int i = startStreamId; i < endStreamId; i++) {
- final String streamName = String.format("%s_%d", streamPrefix, i);
- final DistributedLogManager dlm = namespace.openLog(streamName);
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- try {
- AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
- if (null != writers.putIfAbsent(streamName, writer)) {
- FutureUtils.result(writer.asyncClose());
- }
- latch.countDown();
- } catch (IOException e) {
- LOG.error("Failed to intialize writer for stream : {}", streamName, e);
- }
-
- }
- });
- dlms.add(dlm);
- }
- try {
- latch.await();
- } catch (InterruptedException e) {
- throw new IOException("Interrupted on initializing writers for streams.", e);
- }
- for (int i = startStreamId; i < endStreamId; i++) {
- final String streamName = String.format("%s_%d", streamPrefix, i);
- AsyncLogWriter writer = writers.get(streamName);
- if (null == writer) {
- throw new IOException("Writer for " + streamName + " never initialized.");
- }
- streamWriters.add(writer);
- }
- LOG.info("Writing to {} streams.", numStreams);
- }
-
- void rescueWriter(int idx, AsyncLogWriter writer) {
- if (streamWriters.get(idx) == writer) {
- try {
- FutureUtils.result(writer.asyncClose());
- } catch (IOException e) {
- LOG.error("Failed to close writer for stream {}.", idx);
- }
- AsyncLogWriter newWriter = null;
- try {
- newWriter = dlms.get(idx).startAsyncLogSegmentNonPartitioned();
- } catch (IOException e) {
- LOG.error("Failed to create new writer for stream {}, backoff for {} ms.",
- idx, BACKOFF_MS);
- scheduleRescue(idx, writer, BACKOFF_MS);
- }
- streamWriters.set(idx, newWriter);
- } else {
- LOG.warn("AsyncLogWriter for stream {} was already rescued.", idx);
- }
- }
-
- void scheduleRescue(final int idx, final AsyncLogWriter writer, int delayMs) {
- Runnable r = new Runnable() {
- @Override
- public void run() {
- rescueWriter(idx, writer);
- }
- };
- if (delayMs > 0) {
- rescueService.schedule(r, delayMs, TimeUnit.MILLISECONDS);
- } else {
- rescueService.submit(r);
- }
- }
-
- @Override
- public void close() throws IOException {
- this.running = false;
- SchedulerUtils.shutdownScheduler(this.executorService, 2, TimeUnit.MINUTES);
- SchedulerUtils.shutdownScheduler(this.rescueService, 2, TimeUnit.MINUTES);
- for (AsyncLogWriter writer : streamWriters) {
- FutureUtils.result(writer.asyncClose());
- }
- for (DistributedLogManager dlm : dlms) {
- dlm.close();
- }
- namespace.close();
- }
-
- @Override
- public void run() {
- LOG.info("Starting dlwriter (concurrency = {}, prefix = {}, numStreams = {})",
- new Object[] { writeConcurrency, streamPrefix, numStreams });
- for (int i = 0; i < writeConcurrency; i++) {
- executorService.submit(new Writer(i));
- }
- }
-
- class Writer implements Runnable {
-
- final int idx;
-
- Writer(int idx) {
- this.idx = idx;
- }
-
- @Override
- public void run() {
- LOG.info("Started writer {}.", idx);
- while (running) {
- final int streamIdx = random.nextInt(numStreams);
- final AsyncLogWriter writer = streamWriters.get(streamIdx);
- rateLimiter.getLimiter().acquire();
- final long requestMillis = System.currentTimeMillis();
- final byte[] data;
- try {
- data = Utils.generateMessage(requestMillis, messageSizeBytes);
- } catch (TException e) {
- LOG.error("Error on generating message : ", e);
- break;
- }
- writer.write(new LogRecord(requestMillis, data)).addEventListener(new FutureEventListener<DLSN>() {
- @Override
- public void onSuccess(DLSN value) {
- requestStat.registerSuccessfulEvent(System.currentTimeMillis() - requestMillis);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- requestStat.registerFailedEvent(System.currentTimeMillis() - requestMillis);
- LOG.error("Failed to publish, rescue it : ", cause);
- scheduleRescue(streamIdx, writer, 0);
- }
- });
- }
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/ReaderWorker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/ReaderWorker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/ReaderWorker.java
deleted file mode 100644
index adbdeda..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/ReaderWorker.java
+++ /dev/null
@@ -1,468 +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.benchmark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.AsyncLogReader;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogRecordSet;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.benchmark.thrift.Message;
-import com.twitter.distributedlog.client.serverset.DLZkServerSet;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration$;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import java.io.IOException;
-import java.net.URI;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The benchmark for core library reader.
- */
-public class ReaderWorker implements Worker {
-
- private static final Logger LOG = LoggerFactory.getLogger(ReaderWorker.class);
-
- static final int BACKOFF_MS = 200;
-
- final String streamPrefix;
- final int startStreamId;
- final int endStreamId;
- final ScheduledExecutorService executorService;
- final ExecutorService callbackExecutor;
- final DistributedLogNamespace namespace;
- final DistributedLogManager[] dlms;
- final AsyncLogReader[] logReaders;
- final StreamReader[] streamReaders;
- final int numStreams;
- final boolean readFromHead;
-
- final int truncationIntervalInSeconds;
- // DL Client Related Variables
- final DLZkServerSet[] serverSets;
- final List<String> finagleNames;
- final DistributedLogClient dlc;
-
- volatile boolean running = true;
-
- final StatsReceiver statsReceiver;
- final StatsLogger statsLogger;
- final OpStatsLogger e2eStat;
- final OpStatsLogger deliveryStat;
- final OpStatsLogger negativeE2EStat;
- final OpStatsLogger negativeDeliveryStat;
- final OpStatsLogger truncationStat;
- final Counter invalidRecordsCounter;
- final Counter outOfOrderSequenceIdCounter;
-
- class StreamReader implements FutureEventListener<List<LogRecordWithDLSN>>, Runnable, Gauge<Number> {
-
- final int streamIdx;
- final String streamName;
- DLSN prevDLSN = null;
- long prevSequenceId = Long.MIN_VALUE;
- private static final String gaugeLabel = "sequence_id";
-
- StreamReader(int idx, StatsLogger statsLogger) {
- this.streamIdx = idx;
- int streamId = startStreamId + streamIdx;
- streamName = String.format("%s_%d", streamPrefix, streamId);
- statsLogger.scope(streamName).registerGauge(gaugeLabel, this);
- }
-
- @Override
- public void onSuccess(final List<LogRecordWithDLSN> records) {
- for (final LogRecordWithDLSN record : records) {
- if (record.isRecordSet()) {
- try {
- processRecordSet(record);
- } catch (IOException e) {
- onFailure(e);
- }
- } else {
- processRecord(record);
- }
- }
- readLoop();
- }
-
- public void processRecordSet(final LogRecordWithDLSN record) throws IOException {
- LogRecordSet.Reader reader = LogRecordSet.of(record);
- LogRecordWithDLSN nextRecord = reader.nextRecord();
- while (null != nextRecord) {
- processRecord(nextRecord);
- nextRecord = reader.nextRecord();
- }
- }
-
- public void processRecord(final LogRecordWithDLSN record) {
- Message msg;
- try {
- msg = Utils.parseMessage(record.getPayload());
- } catch (TException e) {
- invalidRecordsCounter.inc();
- LOG.warn("Failed to parse record {} for stream {} : size = {} , ",
- new Object[] { record, streamIdx, record.getPayload().length, e });
- return;
- }
- long curTimeMillis = System.currentTimeMillis();
- long e2eLatency = curTimeMillis - msg.getPublishTime();
- long deliveryLatency = curTimeMillis - record.getTransactionId();
- if (e2eLatency >= 0) {
- e2eStat.registerSuccessfulEvent(e2eLatency);
- } else {
- negativeE2EStat.registerSuccessfulEvent(-e2eLatency);
- }
- if (deliveryLatency >= 0) {
- deliveryStat.registerSuccessfulEvent(deliveryLatency);
- } else {
- negativeDeliveryStat.registerSuccessfulEvent(-deliveryLatency);
- }
-
- prevDLSN = record.getDlsn();
- }
-
- @Override
- public void onFailure(Throwable cause) {
- scheduleReinitStream(streamIdx).map(new Function<Void, Void>() {
- @Override
- public Void apply(Void value) {
- prevDLSN = null;
- prevSequenceId = Long.MIN_VALUE;
- readLoop();
- return null;
- }
- });
- }
-
- void readLoop() {
- if (!running) {
- return;
- }
- logReaders[streamIdx].readBulk(10).addEventListener(this);
- }
-
- @Override
- public void run() {
- final DLSN dlsnToTruncate = prevDLSN;
- if (null == dlsnToTruncate) {
- return;
- }
- final Stopwatch stopwatch = Stopwatch.createStarted();
- dlc.truncate(streamName, dlsnToTruncate).addEventListener(
- new FutureEventListener<Boolean>() {
- @Override
- public void onSuccess(Boolean value) {
- truncationStat.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS));
- }
-
- @Override
- public void onFailure(Throwable cause) {
- truncationStat.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS));
- LOG.error("Failed to truncate stream {} to {} : ",
- new Object[]{streamName, dlsnToTruncate, cause});
- }
- });
- }
-
- @Override
- public Number getDefaultValue() {
- return Long.MIN_VALUE;
- }
-
- @Override
- public synchronized Number getSample() {
- return prevSequenceId;
- }
-
- void unregisterGauge() {
- statsLogger.scope(streamName).unregisterGauge(gaugeLabel, this);
- }
- }
-
- public ReaderWorker(DistributedLogConfiguration conf,
- URI uri,
- String streamPrefix,
- int startStreamId,
- int endStreamId,
- int readThreadPoolSize,
- List<String> serverSetPaths,
- List<String> finagleNames,
- int truncationIntervalInSeconds,
- boolean readFromHead, /* read from the earliest data of log */
- StatsReceiver statsReceiver,
- StatsLogger statsLogger) throws IOException {
- checkArgument(startStreamId <= endStreamId);
- this.streamPrefix = streamPrefix;
- this.startStreamId = startStreamId;
- this.endStreamId = endStreamId;
- this.truncationIntervalInSeconds = truncationIntervalInSeconds;
- this.readFromHead = readFromHead;
- this.statsReceiver = statsReceiver;
- this.statsLogger = statsLogger;
- this.e2eStat = this.statsLogger.getOpStatsLogger("e2e");
- this.negativeE2EStat = this.statsLogger.getOpStatsLogger("e2eNegative");
- this.deliveryStat = this.statsLogger.getOpStatsLogger("delivery");
- this.negativeDeliveryStat = this.statsLogger.getOpStatsLogger("deliveryNegative");
- this.truncationStat = this.statsLogger.getOpStatsLogger("truncation");
- this.invalidRecordsCounter = this.statsLogger.getCounter("invalid_records");
- this.outOfOrderSequenceIdCounter = this.statsLogger.getCounter("out_of_order_seq_id");
- this.executorService = Executors.newScheduledThreadPool(
- readThreadPoolSize, new ThreadFactoryBuilder().setNameFormat("benchmark.reader-%d").build());
- this.callbackExecutor = Executors.newFixedThreadPool(
- Runtime.getRuntime().availableProcessors(),
- new ThreadFactoryBuilder().setNameFormat("benchmark.reader-callback-%d").build());
- this.finagleNames = finagleNames;
- this.serverSets = createServerSets(serverSetPaths);
-
- conf.setDeserializeRecordSetOnReads(false);
-
- if (truncationIntervalInSeconds > 0 && (!finagleNames.isEmpty() || !serverSetPaths.isEmpty())) {
- // Construct client for truncation
- DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
- .clientId(ClientId$.MODULE$.apply("dlog_loadtest_reader"))
- .clientBuilder(ClientBuilder.get()
- .hostConnectionLimit(10)
- .hostConnectionCoresize(10)
- .tcpConnectTimeout(Duration$.MODULE$.fromSeconds(1))
- .requestTimeout(Duration$.MODULE$.fromSeconds(2)))
- .redirectBackoffStartMs(100)
- .redirectBackoffMaxMs(500)
- .requestTimeoutMs(2000)
- .statsReceiver(statsReceiver)
- .thriftmux(true)
- .name("reader");
-
- if (serverSetPaths.isEmpty()) {
- // Prepare finagle names
- String local = finagleNames.get(0);
- String[] remotes = new String[finagleNames.size() - 1];
- finagleNames.subList(1, finagleNames.size()).toArray(remotes);
-
- builder = builder.finagleNameStrs(local, remotes);
- LOG.info("Initialized distributedlog client for truncation @ {}.", finagleNames);
- } else if (serverSets.length != 0){
- ServerSet local = this.serverSets[0].getServerSet();
- ServerSet[] remotes = new ServerSet[this.serverSets.length - 1];
- for (int i = 1; i < serverSets.length; i++) {
- remotes[i - 1] = serverSets[i].getServerSet();
- }
-
- builder = builder.serverSets(local, remotes);
- LOG.info("Initialized distributedlog client for truncation @ {}.", serverSetPaths);
- } else {
- builder = builder.uri(uri);
- LOG.info("Initialized distributedlog client for namespace {}", uri);
- }
- dlc = builder.build();
- } else {
- dlc = null;
- }
-
- // construct the factory
- this.namespace = DistributedLogNamespaceBuilder.newBuilder()
- .conf(conf)
- .uri(uri)
- .statsLogger(statsLogger.scope("dl"))
- .build();
- this.numStreams = endStreamId - startStreamId;
- this.dlms = new DistributedLogManager[numStreams];
- this.logReaders = new AsyncLogReader[numStreams];
- final CountDownLatch latch = new CountDownLatch(numStreams);
- for (int i = 0; i < numStreams; i++) {
- final int idx = i;
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- reinitStream(idx).map(new Function<Void, Void>() {
- @Override
- public Void apply(Void value) {
- LOG.info("Initialized stream reader {}.", idx);
- latch.countDown();
- return null;
- }
- });
- }
- });
- }
- try {
- latch.await();
- } catch (InterruptedException e) {
- throw new DLInterruptedException("Failed to intialize benchmark readers : ", e);
- }
- this.streamReaders = new StreamReader[numStreams];
- for (int i = 0; i < numStreams; i++) {
- streamReaders[i] = new StreamReader(i, statsLogger.scope("perstream"));
- if (truncationIntervalInSeconds > 0) {
- executorService.scheduleWithFixedDelay(streamReaders[i],
- truncationIntervalInSeconds, truncationIntervalInSeconds, TimeUnit.SECONDS);
- }
- }
- LOG.info("Initialized benchmark reader on {} streams {} : [{} - {})",
- new Object[] { numStreams, streamPrefix, startStreamId, endStreamId });
- }
-
- protected DLZkServerSet[] createServerSets(List<String> serverSetPaths) {
- DLZkServerSet[] serverSets = new DLZkServerSet[serverSetPaths.size()];
- for (int i = 0; i < serverSets.length; i++) {
- String serverSetPath = serverSetPaths.get(i);
- serverSets[i] = DLZkServerSet.of(URI.create(serverSetPath), 60000);
- }
- return serverSets;
- }
-
- private Future<Void> reinitStream(int idx) {
- Promise<Void> promise = new Promise<Void>();
- reinitStream(idx, promise);
- return promise;
- }
-
- private void reinitStream(int idx, Promise<Void> promise) {
- int streamId = startStreamId + idx;
- String streamName = String.format("%s_%d", streamPrefix, streamId);
-
- if (logReaders[idx] != null) {
- try {
- FutureUtils.result(logReaders[idx].asyncClose());
- } catch (IOException e) {
- LOG.warn("Failed on closing stream reader {} : ", streamName, e);
- }
- logReaders[idx] = null;
- }
- if (dlms[idx] != null) {
- try {
- dlms[idx].close();
- } catch (IOException e) {
- LOG.warn("Failed on closing dlm {} : ", streamName, e);
- }
- dlms[idx] = null;
- }
-
- try {
- dlms[idx] = namespace.openLog(streamName);
- } catch (IOException ioe) {
- LOG.error("Failed on creating dlm {} : ", streamName, ioe);
- scheduleReinitStream(idx, promise);
- return;
- }
- DLSN lastDLSN;
- if (readFromHead) {
- lastDLSN = DLSN.InitialDLSN;
- } else {
- try {
- lastDLSN = dlms[idx].getLastDLSN();
- } catch (IOException ioe) {
- LOG.error("Failed on getting last dlsn from stream {} : ", streamName, ioe);
- scheduleReinitStream(idx, promise);
- return;
- }
- }
- try {
- logReaders[idx] = dlms[idx].getAsyncLogReader(lastDLSN);
- } catch (IOException ioe) {
- LOG.error("Failed on opening reader for stream {} starting from {} : ",
- new Object[] { streamName, lastDLSN, ioe });
- scheduleReinitStream(idx, promise);
- return;
- }
- LOG.info("Opened reader for stream {}, starting from {}.", streamName, lastDLSN);
- promise.setValue(null);
- }
-
- Future<Void> scheduleReinitStream(int idx) {
- Promise<Void> promise = new Promise<Void>();
- scheduleReinitStream(idx, promise);
- return promise;
- }
-
- void scheduleReinitStream(final int idx, final Promise<Void> promise) {
- executorService.schedule(new Runnable() {
- @Override
- public void run() {
- reinitStream(idx, promise);
- }
- }, BACKOFF_MS, TimeUnit.MILLISECONDS);
- }
-
- @Override
- public void close() throws IOException {
- this.running = false;
- for (AsyncLogReader reader : logReaders) {
- if (null != reader) {
- FutureUtils.result(reader.asyncClose());
- }
- }
- for (DistributedLogManager dlm : dlms) {
- if (null != dlm) {
- dlm.close();
- }
- }
- namespace.close();
- SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
- SchedulerUtils.shutdownScheduler(callbackExecutor, 2, TimeUnit.MINUTES);
- if (this.dlc != null) {
- this.dlc.close();
- }
- for (DLZkServerSet serverSet: serverSets) {
- serverSet.close();
- }
- // Unregister gauges to prevent GC spirals
- for (StreamReader sr : streamReaders) {
- sr.unregisterGauge();
- }
- }
-
- @Override
- public void run() {
- LOG.info("Starting reader (prefix = {}, numStreams = {}).",
- streamPrefix, numStreams);
- for (StreamReader sr : streamReaders) {
- sr.readLoop();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Utils.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Utils.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Utils.java
deleted file mode 100644
index f5c32db..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Utils.java
+++ /dev/null
@@ -1,57 +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.benchmark;
-
-import com.twitter.distributedlog.benchmark.thrift.Message;
-import java.nio.ByteBuffer;
-import java.util.Random;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TMemoryInputTransport;
-
-/**
- * Utils for generating and parsing messages.
- */
-public class Utils {
-
- static final Random RAND = new Random(System.currentTimeMillis());
- static final ThreadLocal<TSerializer> MSG_SERIALIZER =
- new ThreadLocal<TSerializer>() {
- @Override
- public TSerializer initialValue() {
- return new TSerializer(new TBinaryProtocol.Factory());
- }
- };
-
- public static byte[] generateMessage(long requestMillis, int payLoadSize) throws TException {
- byte[] payload = new byte[payLoadSize];
- RAND.nextBytes(payload);
- Message msg = new Message(requestMillis, ByteBuffer.wrap(payload));
- return MSG_SERIALIZER.get().serialize(msg);
- }
-
- public static Message parseMessage(byte[] data) throws TException {
- Message msg = new Message();
- TMemoryInputTransport transport = new TMemoryInputTransport(data);
- TBinaryProtocol protocol = new TBinaryProtocol(transport);
- msg.read(protocol);
- return msg;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Worker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Worker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Worker.java
deleted file mode 100644
index 6c60034..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Worker.java
+++ /dev/null
@@ -1,26 +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.benchmark;
-
-import java.io.Closeable;
-
-/**
- * Worker to run benchmark.
- */
-public interface Worker extends Closeable, Runnable {
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/WriterWorker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/WriterWorker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/WriterWorker.java
deleted file mode 100644
index dc5a6e2..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/WriterWorker.java
+++ /dev/null
@@ -1,387 +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.benchmark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.benchmark.utils.ShiftableRateLimiter;
-import com.twitter.distributedlog.client.DistributedLogMultiStreamWriter;
-import com.twitter.distributedlog.client.serverset.DLZkServerSet;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration$;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import java.io.IOException;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Benchmark for distributedlog proxy client.
- */
-public class WriterWorker implements Worker {
-
- static final Logger LOG = LoggerFactory.getLogger(WriterWorker.class);
-
- final String streamPrefix;
- final int startStreamId;
- final int endStreamId;
- final int writeConcurrency;
- final int messageSizeBytes;
- final int hostConnectionCoreSize;
- final int hostConnectionLimit;
- final ExecutorService executorService;
- final ShiftableRateLimiter rateLimiter;
- final URI dlUri;
- final DLZkServerSet[] serverSets;
- final List<String> finagleNames;
- final Random random;
- final List<String> streamNames;
- final int numStreams;
- final int batchSize;
- final boolean thriftmux;
- final boolean handshakeWithClientInfo;
- final int sendBufferSize;
- final int recvBufferSize;
- final boolean enableBatching;
- final int batchBufferSize;
- final int batchFlushIntervalMicros;
- private final String routingServiceFinagleName;
-
- volatile boolean running = true;
-
- final StatsReceiver statsReceiver;
- final StatsLogger statsLogger;
- final OpStatsLogger requestStat;
- final StatsLogger exceptionsLogger;
- final StatsLogger dlErrorCodeLogger;
-
- // callback thread
- final ExecutorService executor;
-
- public WriterWorker(String streamPrefix,
- URI uri,
- int startStreamId,
- int endStreamId,
- ShiftableRateLimiter rateLimiter,
- int writeConcurrency,
- int messageSizeBytes,
- int batchSize,
- int hostConnectionCoreSize,
- int hostConnectionLimit,
- List<String> serverSetPaths,
- List<String> finagleNames,
- StatsReceiver statsReceiver,
- StatsLogger statsLogger,
- boolean thriftmux,
- boolean handshakeWithClientInfo,
- int sendBufferSize,
- int recvBufferSize,
- boolean enableBatching,
- int batchBufferSize,
- int batchFlushIntervalMicros,
- String routingServiceFinagleName) {
- checkArgument(startStreamId <= endStreamId);
- checkArgument(!finagleNames.isEmpty() || !serverSetPaths.isEmpty());
- this.streamPrefix = streamPrefix;
- this.dlUri = uri;
- this.startStreamId = startStreamId;
- this.endStreamId = endStreamId;
- this.rateLimiter = rateLimiter;
- this.writeConcurrency = writeConcurrency;
- this.messageSizeBytes = messageSizeBytes;
- this.statsReceiver = statsReceiver;
- this.statsLogger = statsLogger;
- this.requestStat = this.statsLogger.getOpStatsLogger("requests");
- this.exceptionsLogger = statsLogger.scope("exceptions");
- this.dlErrorCodeLogger = statsLogger.scope("dl_error_code");
- this.executorService = Executors.newCachedThreadPool();
- this.random = new Random(System.currentTimeMillis());
- this.batchSize = batchSize;
- this.hostConnectionCoreSize = hostConnectionCoreSize;
- this.hostConnectionLimit = hostConnectionLimit;
- this.thriftmux = thriftmux;
- this.handshakeWithClientInfo = handshakeWithClientInfo;
- this.sendBufferSize = sendBufferSize;
- this.recvBufferSize = recvBufferSize;
- this.enableBatching = enableBatching;
- this.batchBufferSize = batchBufferSize;
- this.batchFlushIntervalMicros = batchFlushIntervalMicros;
- this.finagleNames = finagleNames;
- this.serverSets = createServerSets(serverSetPaths);
- this.executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
- this.routingServiceFinagleName = routingServiceFinagleName;
-
- // Streams
- streamNames = new ArrayList<String>(endStreamId - startStreamId);
- for (int i = startStreamId; i < endStreamId; i++) {
- streamNames.add(String.format("%s_%d", streamPrefix, i));
- }
- numStreams = streamNames.size();
- LOG.info("Writing to {} streams : {}", numStreams, streamNames);
- }
-
- protected DLZkServerSet[] createServerSets(List<String> serverSetPaths) {
- DLZkServerSet[] serverSets = new DLZkServerSet[serverSetPaths.size()];
- for (int i = 0; i < serverSets.length; i++) {
- String serverSetPath = serverSetPaths.get(i);
- serverSets[i] = DLZkServerSet.of(URI.create(serverSetPath), 60000);
- }
- return serverSets;
- }
-
- @Override
- public void close() throws IOException {
- this.running = false;
- SchedulerUtils.shutdownScheduler(this.executorService, 2, TimeUnit.MINUTES);
- for (DLZkServerSet serverSet: serverSets) {
- serverSet.close();
- }
- }
-
- private DistributedLogClient buildDlogClient() {
- ClientBuilder clientBuilder = ClientBuilder.get()
- .hostConnectionLimit(hostConnectionLimit)
- .hostConnectionCoresize(hostConnectionCoreSize)
- .tcpConnectTimeout(Duration$.MODULE$.fromMilliseconds(200))
- .connectTimeout(Duration$.MODULE$.fromMilliseconds(200))
- .requestTimeout(Duration$.MODULE$.fromSeconds(10))
- .sendBufferSize(sendBufferSize)
- .recvBufferSize(recvBufferSize);
-
- ClientId clientId = ClientId$.MODULE$.apply("dlog_loadtest_writer");
-
- DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
- .clientId(clientId)
- .clientBuilder(clientBuilder)
- .thriftmux(thriftmux)
- .redirectBackoffStartMs(100)
- .redirectBackoffMaxMs(500)
- .requestTimeoutMs(10000)
- .statsReceiver(statsReceiver)
- .streamNameRegex("^" + streamPrefix + "_[0-9]+$")
- .handshakeWithClientInfo(handshakeWithClientInfo)
- .periodicHandshakeIntervalMs(TimeUnit.SECONDS.toMillis(30))
- .periodicOwnershipSyncIntervalMs(TimeUnit.MINUTES.toMillis(5))
- .periodicDumpOwnershipCache(true)
- .handshakeTracing(true)
- .serverRoutingServiceFinagleNameStr(routingServiceFinagleName)
- .name("writer");
-
- if (!finagleNames.isEmpty()) {
- String local = finagleNames.get(0);
- String[] remotes = new String[finagleNames.size() - 1];
- finagleNames.subList(1, finagleNames.size()).toArray(remotes);
-
- builder = builder.finagleNameStrs(local, remotes);
- } else if (serverSets.length != 0){
- ServerSet local = serverSets[0].getServerSet();
- ServerSet[] remotes = new ServerSet[serverSets.length - 1];
- for (int i = 1; i < serverSets.length; i++) {
- remotes[i - 1] = serverSets[i].getServerSet();
- }
- builder = builder.serverSets(local, remotes);
- } else {
- builder = builder.uri(dlUri);
- }
-
- return builder.build();
- }
-
- ByteBuffer buildBuffer(long requestMillis, int messageSizeBytes) {
- ByteBuffer data;
- try {
- data = ByteBuffer.wrap(Utils.generateMessage(requestMillis, messageSizeBytes));
- return data;
- } catch (TException e) {
- LOG.error("Error generating message : ", e);
- return null;
- }
- }
-
- List<ByteBuffer> buildBufferList(int batchSize, long requestMillis, int messageSizeBytes) {
- ArrayList<ByteBuffer> bufferList = new ArrayList<ByteBuffer>(batchSize);
- for (int i = 0; i < batchSize; i++) {
- ByteBuffer buf = buildBuffer(requestMillis, messageSizeBytes);
- if (null == buf) {
- return null;
- }
- bufferList.add(buf);
- }
- return bufferList;
- }
-
- class TimedRequestHandler implements FutureEventListener<DLSN>, Runnable {
- final String streamName;
- final long requestMillis;
- DLSN dlsn = null;
- Throwable cause = null;
-
- TimedRequestHandler(String streamName,
- long requestMillis) {
- this.streamName = streamName;
- this.requestMillis = requestMillis;
- }
- @Override
- public void onSuccess(DLSN value) {
- dlsn = value;
- executor.submit(this);
- }
- @Override
- public void onFailure(Throwable cause) {
- this.cause = cause;
- executor.submit(this);
- }
-
- @Override
- public void run() {
- if (null != dlsn) {
- requestStat.registerSuccessfulEvent(System.currentTimeMillis() - requestMillis);
- } else {
- LOG.error("Failed to publish to {} : ", streamName, cause);
- requestStat.registerFailedEvent(System.currentTimeMillis() - requestMillis);
- exceptionsLogger.getCounter(cause.getClass().getName()).inc();
- if (cause instanceof DLException) {
- DLException dle = (DLException) cause;
- dlErrorCodeLogger.getCounter(dle.getCode().toString()).inc();
- }
- }
- }
- }
-
- class Writer implements Runnable {
-
- final int idx;
- final DistributedLogClient dlc;
- DistributedLogMultiStreamWriter writer = null;
- final ShiftableRateLimiter limiter;
-
- Writer(int idx) {
- this.idx = idx;
- this.dlc = buildDlogClient();
- if (enableBatching) {
- writer = DistributedLogMultiStreamWriter.newBuilder()
- .client(this.dlc)
- .streams(streamNames)
- .compressionCodec(CompressionCodec.Type.NONE)
- .flushIntervalMicros(batchFlushIntervalMicros)
- .bufferSize(batchBufferSize)
- .firstSpeculativeTimeoutMs(9000)
- .maxSpeculativeTimeoutMs(9000)
- .requestTimeoutMs(10000)
- .speculativeBackoffMultiplier(2)
- .build();
- }
- this.limiter = rateLimiter.duplicate();
- }
-
- @Override
- public void run() {
- LOG.info("Started writer {}.", idx);
- while (running) {
- this.limiter.getLimiter().acquire();
- final String streamName = streamNames.get(random.nextInt(numStreams));
- final long requestMillis = System.currentTimeMillis();
- final ByteBuffer data = buildBuffer(requestMillis, messageSizeBytes);
- if (null == data) {
- break;
- }
- if (null != writer) {
- writer.write(data).addEventListener(
- new TimedRequestHandler(streamName, requestMillis));
- } else {
- dlc.write(streamName, data).addEventListener(
- new TimedRequestHandler(streamName, requestMillis));
- }
- }
- if (null != writer) {
- writer.close();
- }
- dlc.close();
- }
- }
-
- class BulkWriter implements Runnable {
-
- final int idx;
- final DistributedLogClient dlc;
-
- BulkWriter(int idx) {
- this.idx = idx;
- this.dlc = buildDlogClient();
- }
-
- @Override
- public void run() {
- LOG.info("Started writer {}.", idx);
- while (running) {
- rateLimiter.getLimiter().acquire(batchSize);
- String streamName = streamNames.get(random.nextInt(numStreams));
- final long requestMillis = System.currentTimeMillis();
- final List<ByteBuffer> data = buildBufferList(batchSize, requestMillis, messageSizeBytes);
- if (null == data) {
- break;
- }
- List<Future<DLSN>> results = dlc.writeBulk(streamName, data);
- for (Future<DLSN> result : results) {
- result.addEventListener(new TimedRequestHandler(streamName, requestMillis));
- }
- }
- dlc.close();
- }
- }
-
- @Override
- public void run() {
- LOG.info("Starting writer (concurrency = {}, prefix = {}, batchSize = {})",
- new Object[] { writeConcurrency, streamPrefix, batchSize });
- try {
- for (int i = 0; i < writeConcurrency; i++) {
- Runnable writer = null;
- if (batchSize > 0) {
- writer = new BulkWriter(i);
- } else {
- writer = new Writer(i);
- }
- executorService.submit(writer);
- }
- } catch (Throwable t) {
- LOG.error("Unhandled exception caught", t);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/package-info.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/package-info.java
deleted file mode 100644
index 052a661..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Benchmarks for distributedlog.
- */
-package com.twitter.distributedlog.benchmark;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AbstractReaderBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
deleted file mode 100644
index 4d436ee..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
+++ /dev/null
@@ -1,70 +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.benchmark.stream;
-
-import com.twitter.distributedlog.DistributedLogConstants;
-import org.apache.commons.cli.CommandLine;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-abstract class AbstractReaderBenchmark extends StreamBenchmark {
-
- private static final Logger logger = LoggerFactory.getLogger(SyncReaderBenchmark.class);
-
- protected ReadMode readMode = ReadMode.LATEST;
- protected long fromTxId = DistributedLogConstants.INVALID_TXID;
- protected long rewindMs = 0L;
- protected int batchSize = 1;
-
- protected AbstractReaderBenchmark() {
- options.addOption("t", "tx-id", true,
- "Transaction ID to start read from when reading in mode 'position'");
- options.addOption("r", "rewind", true,
- "Time to rewind back to read from when reading in mode 'rewind' (in milliseconds)");
- options.addOption("m", "mode", true,
- "Read Mode : [oldest, latest, rewind, position]");
- options.addOption("b", "batch-size", true, "Read batch size");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) {
- if (cmdline.hasOption("m")) {
- String mode = cmdline.getOptionValue("m");
- try {
- readMode = ReadMode.valueOf(mode.toUpperCase());
- } catch (IllegalArgumentException iae) {
- logger.error("Invalid read mode {}.", mode);
- printUsage();
- System.exit(0);
- }
- } else {
- printUsage();
- System.exit(0);
- }
- if (cmdline.hasOption("t")) {
- fromTxId = Long.parseLong(cmdline.getOptionValue("t"));
- }
- if (cmdline.hasOption("r")) {
- rewindMs = Long.parseLong(cmdline.getOptionValue("r"));
- }
- if (cmdline.hasOption("b")) {
- batchSize = Integer.parseInt(cmdline.getOptionValue("b"));
- }
- logger.info("Start reading from transaction id {}, rewind {} ms.", fromTxId, rewindMs);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AsyncReaderBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
deleted file mode 100644
index 86acdb6..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
+++ /dev/null
@@ -1,158 +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.benchmark.stream;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.AsyncLogReader;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.util.FutureUtils;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Benchmark on {@link com.twitter.distributedlog.AsyncLogReader} reading from a stream.
- */
-public class AsyncReaderBenchmark extends AbstractReaderBenchmark {
-
- private static final Logger logger = LoggerFactory.getLogger(AsyncReaderBenchmark.class);
-
- @Override
- protected void benchmark(DistributedLogNamespace namespace, String logName, StatsLogger statsLogger) {
- DistributedLogManager dlm = null;
- while (null == dlm) {
- try {
- dlm = namespace.openLog(streamName);
- } catch (IOException ioe) {
- logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
- }
- if (null == dlm) {
- try {
- TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
- } catch (InterruptedException e) {
- logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
- streamName, e);
- }
- }
- }
- logger.info("Created dlm for stream {}.", streamName);
-
- // Stats
- OpStatsLogger openReaderStats = statsLogger.getOpStatsLogger("open_reader");
- OpStatsLogger blockingReadStats = statsLogger.getOpStatsLogger("blocking_read");
- Counter readCounter = statsLogger.getCounter("reads");
-
- AsyncLogReader reader = null;
- DLSN lastDLSN = null;
- Long lastTxId = null;
- while (null == reader) {
- // initialize the last txid
- if (null == lastTxId) {
- switch (readMode) {
- case OLDEST:
- lastTxId = 0L;
- lastDLSN = DLSN.InitialDLSN;
- break;
- case LATEST:
- lastTxId = Long.MAX_VALUE;
- try {
- lastDLSN = dlm.getLastDLSN();
- } catch (IOException ioe) {
- continue;
- }
- break;
- case REWIND:
- lastTxId = System.currentTimeMillis() - rewindMs;
- lastDLSN = null;
- break;
- case POSITION:
- lastTxId = fromTxId;
- lastDLSN = null;
- break;
- default:
- logger.warn("Unsupported mode {}", readMode);
- printUsage();
- System.exit(0);
- break;
- }
- logger.info("Reading from transaction id = {}, dlsn = {}", lastTxId, lastDLSN);
- }
- // Open the reader
- Stopwatch stopwatch = Stopwatch.createStarted();
- try {
- if (null == lastDLSN) {
- reader = FutureUtils.result(dlm.openAsyncLogReader(lastTxId));
- } else {
- reader = FutureUtils.result(dlm.openAsyncLogReader(lastDLSN));
- }
- long elapsedMs = stopwatch.elapsed(TimeUnit.MICROSECONDS);
- openReaderStats.registerSuccessfulEvent(elapsedMs);
- logger.info("It took {} ms to position the reader to transaction id = {}, dlsn = {}",
- lastTxId, lastDLSN);
- } catch (IOException ioe) {
- openReaderStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
- logger.warn("Failed to create reader for stream {} reading from tx id = {}, dlsn = {}.",
- new Object[] { streamName, lastTxId, lastDLSN });
- }
- if (null == reader) {
- try {
- TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
- } catch (InterruptedException e) {
- logger.warn("Interrupted from sleep after reader was reassigned null for stream {} : ",
- streamName, e);
- }
- continue;
- }
- List<LogRecordWithDLSN> records;
- stopwatch = Stopwatch.createUnstarted();
- while (true) {
- try {
- stopwatch.start();
- records = FutureUtils.result(reader.readBulk(batchSize));
- long elapsedMicros = stopwatch.stop().elapsed(TimeUnit.MICROSECONDS);
- blockingReadStats.registerSuccessfulEvent(elapsedMicros);
- if (!records.isEmpty()) {
- readCounter.add(records.size());
- LogRecordWithDLSN lastRecord = records.get(records.size() - 1);
- lastTxId = lastRecord.getTransactionId();
- lastDLSN = lastRecord.getDlsn();
- }
- stopwatch.reset();
- } catch (IOException e) {
- logger.warn("Encountered reading record from stream {} : ", streamName, e);
- reader = null;
- break;
- }
- }
- try {
- TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
- } catch (InterruptedException e) {
- logger.warn("Interrupted from sleep while creating reader for stream {} : ",
- streamName, e);
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerBatchReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerBatchReader.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerBatchReader.java
deleted file mode 100644
index 6a11469..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerBatchReader.java
+++ /dev/null
@@ -1,82 +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.benchmark.stream;
-
-import java.util.Enumeration;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Read ledgers in batches.
- */
-public class LedgerBatchReader implements Runnable {
-
- private static final Logger logger = LoggerFactory.getLogger(LedgerBatchReader.class);
-
- private final LedgerHandle lh;
- private final ReadEntryListener readEntryListener;
- private final int batchSize;
-
- public LedgerBatchReader(LedgerHandle lh,
- ReadEntryListener readEntryListener,
- int batchSize) {
- this.lh = lh;
- this.batchSize = batchSize;
- this.readEntryListener = readEntryListener;
- }
-
- @Override
- public void run() {
- long lac = lh.getLastAddConfirmed();
-
- long entryId = 0L;
-
- while (entryId <= lac) {
- long startEntryId = entryId;
- long endEntryId = Math.min(startEntryId + batchSize - 1, lac);
-
- Enumeration<LedgerEntry> entries = null;
- while (null == entries) {
- try {
- entries = lh.readEntries(startEntryId, endEntryId);
- } catch (BKException bke) {
- logger.error("Encountered exceptions on reading [ {} - {} ] ",
- new Object[] { startEntryId, endEntryId, bke });
- } catch (InterruptedException ie) {
- Thread.currentThread().interrupt();
- break;
- }
- }
- if (null == entries) {
- break;
- }
-
- while (entries.hasMoreElements()) {
- LedgerEntry entry = entries.nextElement();
- readEntryListener.onEntryComplete(BKException.Code.OK, lh, entry, null);
- }
-
- entryId = endEntryId + 1;
- }
-
- }
-}
[33/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
deleted file mode 100644
index 2486297..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
+++ /dev/null
@@ -1,1325 +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.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.function.GetLastTxIdFunction;
-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;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.logsegment.RollingPolicy;
-import com.twitter.distributedlog.logsegment.SizeBasedRollingPolicy;
-import com.twitter.distributedlog.logsegment.TimeBasedRollingPolicy;
-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;
-import com.twitter.distributedlog.util.FutureUtils.FutureEventListenerRunnable;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-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.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static com.twitter.distributedlog.impl.ZKLogSegmentFilters.WRITE_HANDLE_FILTER;
-
-/**
- * Log Handler for Writers.
- *
- * <h3>Metrics</h3>
- * All the metrics about log write handler are exposed under scope `segments`.
- * <ul>
- * <li> `segments`/open : opstats. latency characteristics on starting a new log segment.
- * <li> `segments`/close : opstats. latency characteristics on completing an inprogress log segment.
- * <li> `segments`/recover : opstats. latency characteristics on recovering a log segment.
- * <li> `segments`/delete : opstats. latency characteristics on deleting a log segment.
- * </ul>
- */
-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 MaxTxId maxTxId;
- protected final MaxLogSegmentSequenceNo maxLogSegmentSequenceNo;
- protected final boolean validateLogSegmentSequenceNumber;
- protected final int regionId;
- protected final RollingPolicy rollingPolicy;
- protected Future<? extends DistributedLock> lockFuture = null;
- protected final PermitLimiter writeLimiter;
- protected final FeatureProvider featureProvider;
- protected final DynamicDistributedLogConfiguration dynConf;
- protected final MetadataUpdater metadataUpdater;
- // tracking the inprogress log segments
- protected final LinkedList<Long> inprogressLSSNs;
-
- // Fetch LogSegments State: write can continue without full list of log segments while truncation needs
- private final Future<Versioned<List<LogSegmentMetadata>>> fetchForWrite;
- private Future<Versioned<List<LogSegmentMetadata>>> fetchForTruncation;
-
- // Recover Functions
- private final RecoverLogSegmentFunction recoverLogSegmentFunction =
- new RecoverLogSegmentFunction();
- private final AbstractFunction1<List<LogSegmentMetadata>, Future<Long>> recoverLogSegmentsFunction =
- new AbstractFunction1<List<LogSegmentMetadata>, Future<Long>>() {
- @Override
- public Future<Long> apply(List<LogSegmentMetadata> segmentList) {
- LOG.info("Initiating Recovery For {} : {}", getFullyQualifiedName(), segmentList);
- // if lastLedgerRollingTimeMillis is not updated, we set it to now.
- synchronized (BKLogWriteHandler.this) {
- if (lastLedgerRollingTimeMillis < 0) {
- lastLedgerRollingTimeMillis = Utils.nowInMillis();
- }
- }
-
- if (validateLogSegmentSequenceNumber) {
- synchronized (inprogressLSSNs) {
- for (LogSegmentMetadata segment : segmentList) {
- if (segment.isInProgress()) {
- inprogressLSSNs.addLast(segment.getLogSegmentSequenceNumber());
- }
- }
- }
- }
-
- return FutureUtils.processList(segmentList, recoverLogSegmentFunction, scheduler).map(
- GetLastTxIdFunction.INSTANCE);
- }
- };
-
- // Stats
- private final StatsLogger perLogStatsLogger;
- private final OpStatsLogger closeOpStats;
- private final OpStatsLogger openOpStats;
- private final OpStatsLogger recoverOpStats;
- private final OpStatsLogger deleteOpStats;
-
- /**
- * Construct a Bookkeeper journal manager.
- */
- BKLogWriteHandler(LogMetadataForWriter logMetadata,
- DistributedLogConfiguration conf,
- LogStreamMetadataStore streamMetadataStore,
- LogSegmentMetadataCache metadataCache,
- LogSegmentEntryStore entryStore,
- OrderedScheduler scheduler,
- Allocator<LogSegmentEntryWriter, Object> segmentAllocator,
- StatsLogger statsLogger,
- StatsLogger perLogStatsLogger,
- AlertStatsLogger alertStatsLogger,
- String clientId,
- int regionId,
- PermitLimiter writeLimiter,
- FeatureProvider featureProvider,
- DynamicDistributedLogConfiguration dynConf,
- DistributedLock lock /** owned by handler **/) {
- super(logMetadata,
- conf,
- streamMetadataStore,
- metadataCache,
- entryStore,
- scheduler,
- statsLogger,
- alertStatsLogger,
- clientId);
- this.logMetadataForWriter = logMetadata;
- this.logSegmentAllocator = segmentAllocator;
- this.perLogStatsLogger = perLogStatsLogger;
- this.writeLimiter = writeLimiter;
- this.featureProvider = featureProvider;
- this.dynConf = dynConf;
- this.lock = lock;
- this.metadataUpdater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-
- if (conf.getEncodeRegionIDInLogSegmentMetadata()) {
- this.regionId = regionId;
- } else {
- this.regionId = DistributedLogConstants.LOCAL_REGION_ID;
- }
- this.validateLogSegmentSequenceNumber = conf.isLogSegmentSequenceNumberValidationEnabled();
-
- // Construct the max sequence no
- maxLogSegmentSequenceNo = new MaxLogSegmentSequenceNo(logMetadata.getMaxLSSNData());
- inprogressLSSNs = new LinkedList<Long>();
- // Construct the max txn id.
- maxTxId = new MaxTxId(logMetadata.getMaxTxIdData());
-
- // Schedule fetching log segment list in background before we access it.
- // We don't need to watch the log segment list changes for writer, as it manages log segment list.
- fetchForWrite = readLogSegmentsFromStore(
- LogSegmentMetadata.COMPARATOR,
- WRITE_HANDLE_FILTER,
- null);
-
- // Initialize other parameters.
- setLastLedgerRollingTimeMillis(Utils.nowInMillis());
-
- // Rolling Policy
- if (conf.getLogSegmentRollingIntervalMinutes() > 0) {
- rollingPolicy = new TimeBasedRollingPolicy(conf.getLogSegmentRollingIntervalMinutes() * 60 * 1000L);
- } else {
- rollingPolicy = new SizeBasedRollingPolicy(conf.getMaxLogSegmentBytes());
- }
-
- // Stats
- StatsLogger segmentsStatsLogger = statsLogger.scope("segments");
- openOpStats = segmentsStatsLogger.getOpStatsLogger("open");
- closeOpStats = segmentsStatsLogger.getOpStatsLogger("close");
- recoverOpStats = segmentsStatsLogger.getOpStatsLogger("recover");
- deleteOpStats = segmentsStatsLogger.getOpStatsLogger("delete");
- }
-
- private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFetch(
- final Comparator<LogSegmentMetadata> comparator) {
- final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
- fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(promise, cause);
- }
-
- @Override
- public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
- try {
- FutureUtils.setValue(promise, getCachedLogSegments(comparator));
- } catch (UnexpectedException e) {
- FutureUtils.setException(promise, e);
- }
- }
- });
- return promise;
- }
-
- private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFullFetch(
- final Comparator<LogSegmentMetadata> comparator) {
- Future<Versioned<List<LogSegmentMetadata>>> result;
- synchronized (this) {
- if (null == fetchForTruncation) {
- fetchForTruncation = readLogSegmentsFromStore(
- LogSegmentMetadata.COMPARATOR,
- LogSegmentFilter.DEFAULT_FILTER,
- null);
- }
- result = fetchForTruncation;
- }
-
- final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
- result.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(promise, cause);
- }
-
- @Override
- public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
- try {
- FutureUtils.setValue(promise, getCachedLogSegments(comparator));
- } catch (UnexpectedException e) {
- FutureUtils.setException(promise, e);
- }
- }
- });
- return promise;
- }
-
- // Transactional operations for MaxLogSegmentSequenceNo
- void storeMaxSequenceNumber(final Transaction<Object> txn,
- final MaxLogSegmentSequenceNo maxSeqNo,
- final long seqNo,
- final boolean isInprogress) {
- metadataStore.storeMaxLogSegmentSequenceNumber(txn, logMetadata, maxSeqNo.getVersionedData(seqNo),
- new Transaction.OpListener<Version>() {
- @Override
- public void onCommit(Version version) {
- if (validateLogSegmentSequenceNumber) {
- synchronized (inprogressLSSNs) {
- if (isInprogress) {
- inprogressLSSNs.add(seqNo);
- } else {
- inprogressLSSNs.removeFirst();
- }
- }
- }
- maxSeqNo.update(version, seqNo);
- }
-
- @Override
- public void onAbort(Throwable t) {
- // no-op
- }
- });
- }
-
- // Transactional operations for MaxTxId
- void storeMaxTxId(final Transaction<Object> txn,
- final MaxTxId maxTxId,
- final long txId) {
- metadataStore.storeMaxTxnId(txn, logMetadataForWriter, maxTxId.getVersionedData(txId),
- new Transaction.OpListener<Version>() {
- @Override
- public void onCommit(Version version) {
- maxTxId.update(version, txId);
- }
-
- @Override
- public void onAbort(Throwable t) {
- // no-op
- }
- });
- }
-
- // Transactional operations for logsegment
- void writeLogSegment(final Transaction<Object> txn,
- final LogSegmentMetadata metadata) {
- metadataStore.createLogSegment(txn, metadata, new Transaction.OpListener<Void>() {
- @Override
- public void onCommit(Void r) {
- addLogSegmentToCache(metadata.getSegmentName(), metadata);
- }
-
- @Override
- public void onAbort(Throwable t) {
- // no-op
- }
- });
- }
-
- void deleteLogSegment(final Transaction<Object> txn,
- final LogSegmentMetadata metadata) {
- metadataStore.deleteLogSegment(txn, metadata, new Transaction.OpListener<Void>() {
- @Override
- public void onCommit(Void r) {
- removeLogSegmentFromCache(metadata.getSegmentName());
- }
-
- @Override
- public void onAbort(Throwable t) {
- // no-op
- }
- });
- }
-
- /**
- * The caller could call this before any actions, which to hold the lock for
- * the write handler of its whole lifecycle. The lock will only be released
- * when closing the write handler.
- *
- * This method is useful to prevent releasing underlying zookeeper lock during
- * recovering/completing log segments. Releasing underlying zookeeper lock means
- * 1) increase latency when re-lock on starting new log segment. 2) increase the
- * possibility of a stream being re-acquired by other instances.
- *
- * @return future represents the lock result
- */
- Future<? extends DistributedLock> lockHandler() {
- if (null != lockFuture) {
- return lockFuture;
- }
- lockFuture = lock.asyncAcquire();
- return lockFuture;
- }
-
- Future<Void> unlockHandler() {
- if (null != lockFuture) {
- return lock.asyncClose();
- } else {
- return Future.Void();
- }
- }
-
- /**
- * Start a new log segment in a BookKeeper ledger.
- * First ensure that we have the write lock for this journal.
- * Then create a ledger and stream based on that ledger.
- * The ledger id is written to the inprogress znode, so that in the
- * case of a crash, a recovery process can find the ledger we were writing
- * to when we crashed.
- *
- * @param txId First transaction id to be written to the stream
- * @return
- * @throws IOException
- */
- public BKLogSegmentWriter startLogSegment(long txId) throws IOException {
- return startLogSegment(txId, false, false);
- }
-
- /**
- * Start a new log segment in a BookKeeper ledger.
- * First ensure that we have the write lock for this journal.
- * Then create a ledger and stream based on that ledger.
- * The ledger id is written to the inprogress znode, so that in the
- * case of a crash, a recovery process can find the ledger we were writing
- * to when we crashed.
- *
- * @param txId First transaction id to be written to the stream
- * @param bestEffort
- * @param allowMaxTxID
- * allow using max tx id to start log segment
- * @return
- * @throws IOException
- */
- public BKLogSegmentWriter startLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID)
- throws IOException {
- Stopwatch stopwatch = Stopwatch.createStarted();
- boolean success = false;
- try {
- BKLogSegmentWriter writer = doStartLogSegment(txId, bestEffort, allowMaxTxID);
- success = true;
- return writer;
- } finally {
- if (success) {
- openOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- } else {
- openOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- }
- }
- }
-
- protected long assignLogSegmentSequenceNumber() throws IOException {
- // For any active stream we will always make sure that there is at least one
- // active ledger (except when the stream first starts out). Therefore when we
- // see no ledger metadata for a stream, we assume that this is the first ledger
- // in the stream
- long logSegmentSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
- boolean logSegmentsFound = false;
-
- if (LogSegmentMetadata.supportsLogSegmentSequenceNo(conf.getDLLedgerMetadataLayoutVersion())) {
- List<LogSegmentMetadata> ledgerListDesc = getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR);
- Long nextLogSegmentSeqNo = DLUtils.nextLogSegmentSequenceNumber(ledgerListDesc);
-
- if (null == nextLogSegmentSeqNo) {
- logSegmentsFound = false;
- // we don't find last assigned log segment sequence number
- // then we start the log segment with configured FirstLogSegmentSequenceNumber.
- logSegmentSeqNo = conf.getFirstLogSegmentSequenceNumber();
- } else {
- logSegmentsFound = true;
- // latest log segment is assigned with a sequence number, start with next sequence number
- logSegmentSeqNo = nextLogSegmentSeqNo;
- }
- }
-
- // We only skip log segment sequence number validation only when no log segments found &
- // the maximum log segment sequence number is "UNASSIGNED".
- if (!logSegmentsFound &&
- (DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO == maxLogSegmentSequenceNo.getSequenceNumber())) {
- // no ledger seqno stored in /ledgers before
- LOG.info("No max ledger sequence number found while creating log segment {} for {}.",
- logSegmentSeqNo, getFullyQualifiedName());
- } else if (maxLogSegmentSequenceNo.getSequenceNumber() + 1 != logSegmentSeqNo) {
- LOG.warn("Unexpected max log segment sequence number {} for {} : list of cached segments = {}",
- new Object[]{maxLogSegmentSequenceNo.getSequenceNumber(), getFullyQualifiedName(),
- getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)});
- // there is max log segment number recorded there and it isn't match. throw exception.
- throw new DLIllegalStateException("Unexpected max log segment sequence number "
- + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
- + ", expected " + (logSegmentSeqNo - 1));
- }
-
- return logSegmentSeqNo;
- }
-
- protected BKLogSegmentWriter doStartLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID) throws IOException {
- return FutureUtils.result(asyncStartLogSegment(txId, bestEffort, allowMaxTxID));
- }
-
- protected Future<BKLogSegmentWriter> asyncStartLogSegment(final long txId,
- final boolean bestEffort,
- final boolean allowMaxTxID) {
- final Promise<BKLogSegmentWriter> promise = new Promise<BKLogSegmentWriter>();
- try {
- lock.checkOwnershipAndReacquire();
- } catch (LockingException e) {
- FutureUtils.setException(promise, e);
- return promise;
- }
- fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(promise, cause);
- }
-
- @Override
- public void onSuccess(Versioned<List<LogSegmentMetadata>> list) {
- doStartLogSegment(txId, bestEffort, allowMaxTxID, promise);
- }
- });
- return promise;
- }
-
- protected void doStartLogSegment(final long txId,
- final boolean bestEffort,
- final boolean allowMaxTxID,
- final Promise<BKLogSegmentWriter> promise) {
- // validate the tx id
- if ((txId < 0) ||
- (!allowMaxTxID && (txId == DistributedLogConstants.MAX_TXID))) {
- FutureUtils.setException(promise, new IOException("Invalid Transaction Id " + txId));
- return;
- }
-
- long highestTxIdWritten = maxTxId.get();
- if (txId < highestTxIdWritten) {
- if (highestTxIdWritten == DistributedLogConstants.MAX_TXID) {
- LOG.error("We've already marked the stream as ended and attempting to start a new log segment");
- FutureUtils.setException(promise, new EndOfStreamException("Writing to a stream after it has been marked as completed"));
- return;
- } else {
- LOG.error("We've already seen TxId {} the max TXId is {}", txId, highestTxIdWritten);
- FutureUtils.setException(promise, new TransactionIdOutOfOrderException(txId, highestTxIdWritten));
- return;
- }
- }
-
- try {
- logSegmentAllocator.allocate();
- } catch (IOException e) {
- // failed to issue an allocation request
- failStartLogSegment(promise, bestEffort, e);
- return;
- }
-
- // start the transaction from zookeeper
- final Transaction<Object> txn = streamMetadataStore.newTransaction();
-
- // failpoint injected before creating ledger
- try {
- FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_StartLogSegmentBeforeLedgerCreate);
- } catch (IOException ioe) {
- failStartLogSegment(promise, bestEffort, ioe);
- return;
- }
-
- logSegmentAllocator.tryObtain(txn, NULL_OP_LISTENER)
- .addEventListener(new FutureEventListener<LogSegmentEntryWriter>() {
-
- @Override
- public void onSuccess(LogSegmentEntryWriter entryWriter) {
- // try-obtain succeed
- createInprogressLogSegment(
- txn,
- txId,
- entryWriter,
- bestEffort,
- promise);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- failStartLogSegment(promise, bestEffort, cause);
- }
- });
- }
-
- private void failStartLogSegment(Promise<BKLogSegmentWriter> promise,
- boolean bestEffort,
- Throwable cause) {
- if (bestEffort) {
- FutureUtils.setValue(promise, null);
- } else {
- FutureUtils.setException(promise, cause);
- }
- }
-
- // once the ledger handle is obtained from allocator, this function should guarantee
- // either the transaction is executed or aborted. Otherwise, the ledger handle will
- // just leak from the allocation pool - hence cause "No Ledger Allocator"
- private void createInprogressLogSegment(Transaction<Object> txn,
- final long txId,
- final LogSegmentEntryWriter entryWriter,
- boolean bestEffort,
- final Promise<BKLogSegmentWriter> promise) {
- final long logSegmentSeqNo;
- try {
- FailpointUtils.checkFailPoint(
- FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber);
- logSegmentSeqNo = assignLogSegmentSequenceNumber();
- } catch (IOException e) {
- // abort the current prepared transaction
- txn.abort(e);
- failStartLogSegment(promise, bestEffort, e);
- return;
- }
-
- final String inprogressZnodePath = inprogressZNode(
- entryWriter.getLogSegmentId(), txId, logSegmentSeqNo);
- final LogSegmentMetadata l =
- new LogSegmentMetadata.LogSegmentMetadataBuilder(inprogressZnodePath,
- conf.getDLLedgerMetadataLayoutVersion(), entryWriter.getLogSegmentId(), txId)
- .setLogSegmentSequenceNo(logSegmentSeqNo)
- .setRegionId(regionId)
- .setEnvelopeEntries(
- LogSegmentMetadata.supportsEnvelopedEntries(conf.getDLLedgerMetadataLayoutVersion()))
- .build();
-
- // Create an inprogress segment
- writeLogSegment(txn, l);
-
- // Try storing max sequence number.
- LOG.debug("Try storing max sequence number in startLogSegment {} : {}", inprogressZnodePath, logSegmentSeqNo);
- storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, logSegmentSeqNo, true);
-
- // Try storing max tx id.
- LOG.debug("Try storing MaxTxId in startLogSegment {} {}", inprogressZnodePath, txId);
- storeMaxTxId(txn, maxTxId, txId);
-
- txn.execute().addEventListener(FutureEventListenerRunnable.of(new FutureEventListener<Void>() {
-
- @Override
- public void onSuccess(Void value) {
- try {
- FutureUtils.setValue(promise, new BKLogSegmentWriter(
- getFullyQualifiedName(),
- l.getSegmentName(),
- conf,
- conf.getDLLedgerMetadataLayoutVersion(),
- entryWriter,
- lock,
- txId,
- logSegmentSeqNo,
- scheduler,
- statsLogger,
- perLogStatsLogger,
- alertStatsLogger,
- writeLimiter,
- featureProvider,
- dynConf));
- } catch (IOException ioe) {
- failStartLogSegment(promise, false, ioe);
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- failStartLogSegment(promise, false, cause);
- }
- }, scheduler));
- }
-
- boolean shouldStartNewSegment(BKLogSegmentWriter writer) {
- return rollingPolicy.shouldRollover(writer, lastLedgerRollingTimeMillis);
- }
-
- /**
- * Finalize a log segment. If the journal manager is currently
- * writing to a ledger, ensure that this is the ledger of the log segment
- * being finalized.
- * <p/>
- * Otherwise this is the recovery case. In the recovery case, ensure that
- * the firstTxId of the ledger matches firstTxId for the segment we are
- * trying to finalize.
- */
- Future<LogSegmentMetadata> completeAndCloseLogSegment(final BKLogSegmentWriter writer) {
- final Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
- completeAndCloseLogSegment(writer, promise);
- return promise;
- }
-
- private void completeAndCloseLogSegment(final BKLogSegmentWriter writer,
- final Promise<LogSegmentMetadata> promise) {
- writer.asyncClose().addEventListener(new FutureEventListener<Void>() {
- @Override
- public void onSuccess(Void value) {
- // in theory closeToFinalize should throw exception if a stream is in error.
- // just in case, add another checking here to make sure we don't close log segment is a stream is in error.
- if (writer.shouldFailCompleteLogSegment()) {
- FutureUtils.setException(promise,
- new IOException("LogSegmentWriter for " + writer.getFullyQualifiedLogSegment() + " is already in error."));
- return;
- }
- doCompleteAndCloseLogSegment(
- inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
- writer.getLogSegmentSequenceNumber(),
- writer.getLogSegmentId(),
- writer.getStartTxId(),
- writer.getLastTxId(),
- writer.getPositionWithinLogSegment(),
- writer.getLastDLSN().getEntryId(),
- writer.getLastDLSN().getSlotId(),
- promise);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(promise, cause);
- }
- });
- }
-
- @VisibleForTesting
- LogSegmentMetadata completeAndCloseLogSegment(long logSegmentSeqNo,
- long logSegmentId,
- long firstTxId,
- long lastTxId,
- int recordCount)
- throws IOException {
- return completeAndCloseLogSegment(inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo), logSegmentSeqNo,
- logSegmentId, firstTxId, lastTxId, recordCount, -1, -1);
- }
-
- /**
- * Finalize a log segment. If the journal manager is currently
- * writing to a ledger, ensure that this is the ledger of the log segment
- * being finalized.
- * <p/>
- * Otherwise this is the recovery case. In the recovery case, ensure that
- * the firstTxId of the ledger matches firstTxId for the segment we are
- * trying to finalize.
- */
- LogSegmentMetadata completeAndCloseLogSegment(String inprogressZnodeName, long logSegmentSeqNo,
- long logSegmentId, long firstTxId, long lastTxId,
- int recordCount, long lastEntryId, long lastSlotId)
- throws IOException {
- Stopwatch stopwatch = Stopwatch.createStarted();
- boolean success = false;
- try {
- LogSegmentMetadata completedLogSegment =
- doCompleteAndCloseLogSegment(inprogressZnodeName, logSegmentSeqNo,
- logSegmentId, firstTxId, lastTxId, recordCount,
- lastEntryId, lastSlotId);
- success = true;
- return completedLogSegment;
- } finally {
- if (success) {
- closeOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- } else {
- closeOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- }
- }
- }
-
- protected long computeStartSequenceId(LogSegmentMetadata segment) throws IOException {
- if (!segment.isInProgress()) {
- return segment.getStartSequenceId();
- }
-
- long startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
-
- // we only record sequence id when both write version and logsegment's version support sequence id
- if (LogSegmentMetadata.supportsSequenceId(conf.getDLLedgerMetadataLayoutVersion())
- && segment.supportsSequenceId()) {
- List<LogSegmentMetadata> logSegmentDescList =
- getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR);
- startSequenceId = DLUtils.computeStartSequenceId(logSegmentDescList, segment);
- }
-
- return startSequenceId;
- }
-
- /**
- * Close log segment
- *
- * @param inprogressZnodeName
- * @param logSegmentSeqNo
- * @param logSegmentId
- * @param firstTxId
- * @param lastTxId
- * @param recordCount
- * @param lastEntryId
- * @param lastSlotId
- * @throws IOException
- */
- protected LogSegmentMetadata doCompleteAndCloseLogSegment(
- String inprogressZnodeName,
- long logSegmentSeqNo,
- long logSegmentId,
- long firstTxId,
- long lastTxId,
- int recordCount,
- long lastEntryId,
- long lastSlotId) throws IOException {
- Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
- doCompleteAndCloseLogSegment(
- inprogressZnodeName,
- logSegmentSeqNo,
- logSegmentId,
- firstTxId,
- lastTxId,
- recordCount,
- lastEntryId,
- lastSlotId,
- promise);
- return FutureUtils.result(promise);
- }
-
- protected void doCompleteAndCloseLogSegment(final String inprogressZnodeName,
- final long logSegmentSeqNo,
- final long logSegmentId,
- final long firstTxId,
- final long lastTxId,
- final int recordCount,
- final long lastEntryId,
- final long lastSlotId,
- final Promise<LogSegmentMetadata> promise) {
- fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(promise, cause);
- }
-
- @Override
- public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
- doCompleteAndCloseLogSegmentAfterLogSegmentListFetched(
- inprogressZnodeName,
- logSegmentSeqNo,
- logSegmentId,
- firstTxId,
- lastTxId,
- recordCount,
- lastEntryId,
- lastSlotId,
- promise);
- }
- });
- }
-
- private void doCompleteAndCloseLogSegmentAfterLogSegmentListFetched(
- final String inprogressZnodeName,
- long logSegmentSeqNo,
- long logSegmentId,
- long firstTxId,
- long lastTxId,
- int recordCount,
- long lastEntryId,
- long lastSlotId,
- final Promise<LogSegmentMetadata> promise) {
- try {
- lock.checkOwnershipAndReacquire();
- } catch (IOException ioe) {
- FutureUtils.setException(promise, ioe);
- return;
- }
-
- LOG.debug("Completing and Closing Log Segment {} {}", firstTxId, lastTxId);
- LogSegmentMetadata inprogressLogSegment = readLogSegmentFromCache(inprogressZnodeName);
-
- // validate log segment
- if (inprogressLogSegment.getLogSegmentId() != logSegmentId) {
- FutureUtils.setException(promise, new IOException(
- "Active ledger has different ID to inprogress. "
- + inprogressLogSegment.getLogSegmentId() + " found, "
- + logSegmentId + " expected"));
- return;
- }
- // validate the transaction id
- if (inprogressLogSegment.getFirstTxId() != firstTxId) {
- FutureUtils.setException(promise, new IOException("Transaction id not as expected, "
- + inprogressLogSegment.getFirstTxId() + " found, " + firstTxId + " expected"));
- return;
- }
- // validate the log sequence number
- if (validateLogSegmentSequenceNumber) {
- synchronized (inprogressLSSNs) {
- if (inprogressLSSNs.isEmpty()) {
- FutureUtils.setException(promise, new UnexpectedException(
- "Didn't find matched inprogress log segments when completing inprogress "
- + inprogressLogSegment));
- return;
- }
- long leastInprogressLSSN = inprogressLSSNs.getFirst();
- // the log segment sequence number in metadata {@link inprogressLogSegment.getLogSegmentSequenceNumber()}
- // should be same as the sequence number we are completing (logSegmentSeqNo)
- // and
- // it should also be same as the least inprogress log segment sequence number tracked in {@link inprogressLSSNs}
- if ((inprogressLogSegment.getLogSegmentSequenceNumber() != logSegmentSeqNo) ||
- (leastInprogressLSSN != logSegmentSeqNo)) {
- FutureUtils.setException(promise, new UnexpectedException(
- "Didn't find matched inprogress log segments when completing inprogress "
- + inprogressLogSegment));
- return;
- }
- }
- }
-
- // store max sequence number.
- long maxSeqNo= Math.max(logSegmentSeqNo, maxLogSegmentSequenceNo.getSequenceNumber());
- if (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo ||
- (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo + 1)) {
- // ignore the case that a new inprogress log segment is pre-allocated
- // before completing current inprogress one
- LOG.info("Try storing max sequence number {} in completing {}.",
- new Object[] { logSegmentSeqNo, inprogressLogSegment.getZkPath() });
- } else {
- LOG.warn("Unexpected max ledger sequence number {} found while completing log segment {} for {}",
- new Object[] { maxLogSegmentSequenceNo.getSequenceNumber(), logSegmentSeqNo, getFullyQualifiedName() });
- if (validateLogSegmentSequenceNumber) {
- FutureUtils.setException(promise, new DLIllegalStateException("Unexpected max log segment sequence number "
- + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
- + ", expected " + (logSegmentSeqNo - 1)));
- return;
- }
- }
-
- // Prepare the completion
- final String pathForCompletedLedger = completedLedgerZNode(firstTxId, lastTxId, logSegmentSeqNo);
- long startSequenceId;
- try {
- startSequenceId = computeStartSequenceId(inprogressLogSegment);
- } catch (IOException ioe) {
- FutureUtils.setException(promise, ioe);
- return;
- }
- // write completed ledger znode
- final LogSegmentMetadata completedLogSegment =
- inprogressLogSegment.completeLogSegment(
- pathForCompletedLedger,
- lastTxId,
- recordCount,
- lastEntryId,
- lastSlotId,
- startSequenceId);
- setLastLedgerRollingTimeMillis(completedLogSegment.getCompletionTime());
-
- // prepare the transaction
- Transaction<Object> txn = streamMetadataStore.newTransaction();
-
- // create completed log segment
- writeLogSegment(txn, completedLogSegment);
- // delete inprogress log segment
- deleteLogSegment(txn, inprogressLogSegment);
- // store max sequence number
- storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, maxSeqNo, false);
- // update max txn id.
- LOG.debug("Trying storing LastTxId in Finalize Path {} LastTxId {}", pathForCompletedLedger, lastTxId);
- storeMaxTxId(txn, maxTxId, lastTxId);
-
- txn.execute().addEventListener(FutureEventListenerRunnable.of(new FutureEventListener<Void>() {
- @Override
- public void onSuccess(Void value) {
- LOG.info("Completed {} to {} for {} : {}",
- new Object[] { inprogressZnodeName, completedLogSegment.getSegmentName(),
- getFullyQualifiedName(), completedLogSegment });
- FutureUtils.setValue(promise, completedLogSegment);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(promise, cause);
- }
- }, scheduler));
- }
-
- public Future<Long> recoverIncompleteLogSegments() {
- try {
- FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_RecoverIncompleteLogSegments);
- } catch (IOException ioe) {
- return Future.exception(ioe);
- }
- return getCachedLogSegmentsAfterFirstFetch(LogSegmentMetadata.COMPARATOR).flatMap(recoverLogSegmentsFunction);
- }
-
- class RecoverLogSegmentFunction extends Function<LogSegmentMetadata, Future<LogSegmentMetadata>> {
-
- @Override
- public Future<LogSegmentMetadata> apply(final LogSegmentMetadata l) {
- if (!l.isInProgress()) {
- return Future.value(l);
- }
-
- LOG.info("Recovering last record in log segment {} for {}.", l, getFullyQualifiedName());
- return asyncReadLastRecord(l, true, true, true).flatMap(
- new AbstractFunction1<LogRecordWithDLSN, Future<LogSegmentMetadata>>() {
- @Override
- public Future<LogSegmentMetadata> apply(LogRecordWithDLSN lastRecord) {
- return completeLogSegment(l, lastRecord);
- }
- });
- }
-
- private Future<LogSegmentMetadata> completeLogSegment(LogSegmentMetadata l,
- LogRecordWithDLSN lastRecord) {
- LOG.info("Recovered last record in log segment {} for {}.", l, getFullyQualifiedName());
-
- long endTxId = DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID;
- int recordCount = 0;
- long lastEntryId = -1;
- long lastSlotId = -1;
-
- if (null != lastRecord) {
- endTxId = lastRecord.getTransactionId();
- recordCount = lastRecord.getLastPositionWithinLogSegment();
- lastEntryId = lastRecord.getDlsn().getEntryId();
- lastSlotId = lastRecord.getDlsn().getSlotId();
- }
-
- if (endTxId == DistributedLogConstants.INVALID_TXID) {
- LOG.error("Unrecoverable corruption has occurred in segment "
- + l.toString() + " at path " + l.getZkPath()
- + ". Unable to continue recovery.");
- return Future.exception(new IOException("Unrecoverable corruption,"
- + " please check logs."));
- } else if (endTxId == DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID) {
- // TODO: Empty ledger - Ideally we should just remove it?
- endTxId = l.getFirstTxId();
- }
-
- Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
- doCompleteAndCloseLogSegment(
- l.getZNodeName(),
- l.getLogSegmentSequenceNumber(),
- l.getLogSegmentId(),
- l.getFirstTxId(),
- endTxId,
- recordCount,
- lastEntryId,
- lastSlotId,
- promise);
- return promise;
- }
-
- }
-
- Future<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(final DLSN dlsn) {
- if (DLSN.InvalidDLSN == dlsn) {
- List<LogSegmentMetadata> emptyList = new ArrayList<LogSegmentMetadata>(0);
- return Future.value(emptyList);
- }
- return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
- new AbstractFunction1<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
- @Override
- public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
- return setLogSegmentsOlderThanDLSNTruncated(logSegments, dlsn);
- }
- });
- }
-
- private Future<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(List<LogSegmentMetadata> logSegments,
- final DLSN dlsn) {
- LOG.debug("Setting truncation status on logs older than {} from {} for {}",
- new Object[]{dlsn, logSegments, getFullyQualifiedName()});
- List<LogSegmentMetadata> truncateList = new ArrayList<LogSegmentMetadata>(logSegments.size());
- LogSegmentMetadata partialTruncate = null;
- LOG.info("{}: Truncating log segments older than {}", getFullyQualifiedName(), dlsn);
- for (int i = 0; i < logSegments.size(); i++) {
- LogSegmentMetadata l = logSegments.get(i);
- if (!l.isInProgress()) {
- if (l.getLastDLSN().compareTo(dlsn) < 0) {
- LOG.debug("{}: Truncating log segment {} ", getFullyQualifiedName(), l);
- truncateList.add(l);
- } else if (l.getFirstDLSN().compareTo(dlsn) < 0) {
- // Can be satisfied by at most one segment
- if (null != partialTruncate) {
- String logMsg = String.format("Potential metadata inconsistency for stream %s at segment %s", getFullyQualifiedName(), l);
- LOG.error(logMsg);
- return Future.exception(new DLIllegalStateException(logMsg));
- }
- LOG.info("{}: Partially truncating log segment {} older than {}.", new Object[] {getFullyQualifiedName(), l, dlsn});
- partialTruncate = l;
- } else {
- break;
- }
- } else {
- break;
- }
- }
- return setLogSegmentTruncationStatus(truncateList, partialTruncate, dlsn);
- }
-
- private int getNumCandidateLogSegmentsToPurge(List<LogSegmentMetadata> logSegments) {
- if (logSegments.isEmpty()) {
- return 0;
- } else {
- // we have to keep at least one completed log segment for sequence id
- int numCandidateLogSegments = 0;
- for (LogSegmentMetadata segment : logSegments) {
- if (segment.isInProgress()) {
- break;
- } else {
- ++numCandidateLogSegments;
- }
- }
-
- return numCandidateLogSegments - 1;
- }
- }
-
- Future<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTimestamp(final long minTimestampToKeep) {
- if (minTimestampToKeep >= Utils.nowInMillis()) {
- return Future.exception(new IllegalArgumentException(
- "Invalid timestamp " + minTimestampToKeep + " to purge logs for " + getFullyQualifiedName()));
- }
- return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
- new Function<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
- @Override
- public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
- List<LogSegmentMetadata> purgeList = new ArrayList<LogSegmentMetadata>(logSegments.size());
-
- int numCandidates = getNumCandidateLogSegmentsToPurge(logSegments);
-
- for (int iterator = 0; iterator < numCandidates; iterator++) {
- LogSegmentMetadata l = logSegments.get(iterator);
- // When application explicitly truncates segments; timestamp based purge is
- // only used to cleanup log segments that have been marked for truncation
- if ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) &&
- !l.isInProgress() && (l.getCompletionTime() < minTimestampToKeep)) {
- purgeList.add(l);
- } else {
- // stop truncating log segments if we find either an inprogress or a partially
- // truncated log segment
- break;
- }
- }
- LOG.info("Deleting log segments older than {} for {} : {}",
- new Object[] { minTimestampToKeep, getFullyQualifiedName(), purgeList });
- return deleteLogSegments(purgeList);
- }
- });
- }
-
- Future<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTxnId(final long minTxIdToKeep) {
- return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
- new AbstractFunction1<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
- @Override
- public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
- int numLogSegmentsToProcess;
-
- if (minTxIdToKeep < 0) {
- // we are deleting the log, we can remove whole log segments
- numLogSegmentsToProcess = logSegments.size();
- } else {
- numLogSegmentsToProcess = getNumCandidateLogSegmentsToPurge(logSegments);
- }
- List<LogSegmentMetadata> purgeList = Lists.newArrayListWithExpectedSize(numLogSegmentsToProcess);
- for (int iterator = 0; iterator < numLogSegmentsToProcess; iterator++) {
- LogSegmentMetadata l = logSegments.get(iterator);
- if ((minTxIdToKeep < 0) ||
- ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) &&
- !l.isInProgress() && (l.getLastTxId() < minTxIdToKeep))) {
- purgeList.add(l);
- } else {
- // stop truncating log segments if we find either an inprogress or a partially
- // truncated log segment
- break;
- }
- }
- return deleteLogSegments(purgeList);
- }
- });
- }
-
- private Future<List<LogSegmentMetadata>> setLogSegmentTruncationStatus(
- final List<LogSegmentMetadata> truncateList,
- LogSegmentMetadata partialTruncate,
- DLSN minActiveDLSN) {
- final List<LogSegmentMetadata> listToTruncate = Lists.newArrayListWithCapacity(truncateList.size() + 1);
- final List<LogSegmentMetadata> listAfterTruncated = Lists.newArrayListWithCapacity(truncateList.size() + 1);
- Transaction<Object> updateTxn = metadataUpdater.transaction();
- for(LogSegmentMetadata l : truncateList) {
- if (!l.isTruncated()) {
- LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentTruncated(updateTxn, l);
- listToTruncate.add(l);
- listAfterTruncated.add(newSegment);
- }
- }
-
- if (null != partialTruncate && (partialTruncate.isNonTruncated() ||
- (partialTruncate.isPartiallyTruncated() && (partialTruncate.getMinActiveDLSN().compareTo(minActiveDLSN) < 0)))) {
- LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentPartiallyTruncated(
- updateTxn, partialTruncate, minActiveDLSN);
- listToTruncate.add(partialTruncate);
- listAfterTruncated.add(newSegment);
- }
-
- return updateTxn.execute().map(new AbstractFunction1<Void, List<LogSegmentMetadata>>() {
- @Override
- public List<LogSegmentMetadata> apply(Void value) {
- for (int i = 0; i < listToTruncate.size(); i++) {
- removeLogSegmentFromCache(listToTruncate.get(i).getSegmentName());
- LogSegmentMetadata newSegment = listAfterTruncated.get(i);
- addLogSegmentToCache(newSegment.getSegmentName(), newSegment);
- }
- return listAfterTruncated;
- }
- });
- }
-
- private Future<List<LogSegmentMetadata>> deleteLogSegments(
- final List<LogSegmentMetadata> logs) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("Purging logs for {} : {}", getFullyQualifiedName(), logs);
- }
- return FutureUtils.processList(logs,
- new Function<LogSegmentMetadata, Future<LogSegmentMetadata>>() {
- @Override
- public Future<LogSegmentMetadata> apply(LogSegmentMetadata segment) {
- return deleteLogSegment(segment);
- }
- }, scheduler);
- }
-
- private Future<LogSegmentMetadata> deleteLogSegment(
- final LogSegmentMetadata ledgerMetadata) {
- LOG.info("Deleting ledger {} for {}", ledgerMetadata, getFullyQualifiedName());
- final Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
- final Stopwatch stopwatch = Stopwatch.createStarted();
- promise.addEventListener(new FutureEventListener<LogSegmentMetadata>() {
- @Override
- public void onSuccess(LogSegmentMetadata segment) {
- deleteOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- }
-
- @Override
- public void onFailure(Throwable cause) {
- deleteOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- }
- });
- entryStore.deleteLogSegment(ledgerMetadata)
- .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(promise, cause);
- }
-
- @Override
- public void onSuccess(LogSegmentMetadata segment) {
- deleteLogSegmentMetadata(segment, promise);
- }
- });
- return promise;
- }
-
- private void deleteLogSegmentMetadata(final LogSegmentMetadata segmentMetadata,
- final Promise<LogSegmentMetadata> promise) {
- Transaction<Object> deleteTxn = metadataStore.transaction();
- metadataStore.deleteLogSegment(deleteTxn, segmentMetadata, new Transaction.OpListener<Void>() {
- @Override
- public void onCommit(Void r) {
- // purge log segment
- removeLogSegmentFromCache(segmentMetadata.getZNodeName());
- promise.setValue(segmentMetadata);
- }
-
- @Override
- public void onAbort(Throwable t) {
- if (t instanceof LogSegmentNotFoundException) {
- // purge log segment
- removeLogSegmentFromCache(segmentMetadata.getZNodeName());
- promise.setValue(segmentMetadata);
- return;
- } else {
- LOG.error("Couldn't purge {} for {}: with error {}",
- new Object[]{ segmentMetadata, getFullyQualifiedName(), t });
- promise.setException(t);
- }
- }
- });
- deleteTxn.execute();
- }
-
- @Override
- public Future<Void> asyncClose() {
- return Utils.closeSequence(scheduler,
- lock,
- logSegmentAllocator);
- }
-
- @Override
- public Future<Void> asyncAbort() {
- return asyncClose();
- }
-
- String completedLedgerZNodeName(long firstTxId, long lastTxId, long logSegmentSeqNo) {
- if (DistributedLogConstants.LOGSEGMENT_NAME_VERSION == conf.getLogSegmentNameVersion()) {
- return String.format("%s_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, logSegmentSeqNo);
- } else {
- return String.format("%s_%018d_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX,
- firstTxId, lastTxId);
- }
- }
-
- /**
- * Get the znode path for a finalize ledger
- */
- String completedLedgerZNode(long firstTxId, long lastTxId, long logSegmentSeqNo) {
- return String.format("%s/%s", logMetadata.getLogSegmentsPath(),
- completedLedgerZNodeName(firstTxId, lastTxId, logSegmentSeqNo));
- }
-
- /**
- * Get the name of the inprogress znode.
- *
- * @return name of the inprogress znode.
- */
- String inprogressZNodeName(long logSegmentId, long firstTxId, long logSegmentSeqNo) {
- if (DistributedLogConstants.LOGSEGMENT_NAME_VERSION == conf.getLogSegmentNameVersion()) {
- // Lots of the problems are introduced due to different inprogress names with same ledger sequence number.
- return String.format("%s_%018d", DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX, logSegmentSeqNo);
- } else {
- return DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX + "_" + Long.toString(firstTxId, 16);
- }
- }
-
- /**
- * Get the znode path for the inprogressZNode
- */
- String inprogressZNode(long logSegmentId, long firstTxId, long logSegmentSeqNo) {
- return logMetadata.getLogSegmentsPath() + "/" + inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo);
- }
-
- String inprogressZNode(String inprogressZNodeName) {
- return logMetadata.getLogSegmentsPath() + "/" + inprogressZNodeName;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogReader.java
deleted file mode 100644
index 308f42a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogReader.java
+++ /dev/null
@@ -1,276 +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.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Ticker;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.IdleReaderException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.versioning.Versioned;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Synchronous Log Reader based on {@link AsyncLogReader}
- */
-class BKSyncLogReader implements LogReader, AsyncNotification {
-
- private final BKDistributedLogManager bkdlm;
- private final BKLogReadHandler readHandler;
- private final AtomicReference<IOException> readerException =
- new AtomicReference<IOException>(null);
- private final int maxReadAheadWaitTime;
- private Promise<Void> closeFuture;
- private final Optional<Long> startTransactionId;
- private boolean positioned = false;
- private Entry.Reader currentEntry = null;
-
- // readahead reader
- ReadAheadEntryReader readAheadReader = null;
-
- // idle reader settings
- private final boolean shouldCheckIdleReader;
- private final int idleErrorThresholdMillis;
-
- // Stats
- private final Counter idleReaderError;
-
- BKSyncLogReader(DistributedLogConfiguration conf,
- BKDistributedLogManager bkdlm,
- DLSN startDLSN,
- Optional<Long> startTransactionId,
- StatsLogger statsLogger) throws IOException {
- this.bkdlm = bkdlm;
- this.readHandler = bkdlm.createReadHandler(
- Optional.<String>absent(),
- this,
- true);
- this.maxReadAheadWaitTime = conf.getReadAheadWaitTime();
- this.idleErrorThresholdMillis = conf.getReaderIdleErrorThresholdMillis();
- this.shouldCheckIdleReader = idleErrorThresholdMillis > 0 && idleErrorThresholdMillis < Integer.MAX_VALUE;
- this.startTransactionId = startTransactionId;
-
- // start readahead
- startReadAhead(startDLSN);
- if (!startTransactionId.isPresent()) {
- positioned = true;
- }
-
- // Stats
- StatsLogger syncReaderStatsLogger = statsLogger.scope("sync_reader");
- idleReaderError = syncReaderStatsLogger.getCounter("idle_reader_error");
- }
-
- private void startReadAhead(DLSN startDLSN) throws IOException {
- readAheadReader = new ReadAheadEntryReader(
- bkdlm.getStreamName(),
- startDLSN,
- bkdlm.getConf(),
- readHandler,
- bkdlm.getReaderEntryStore(),
- bkdlm.getScheduler(),
- Ticker.systemTicker(),
- bkdlm.alertStatsLogger);
- readHandler.registerListener(readAheadReader);
- readHandler.asyncStartFetchLogSegments()
- .map(new AbstractFunction1<Versioned<List<LogSegmentMetadata>>, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Versioned<List<LogSegmentMetadata>> logSegments) {
- readAheadReader.addStateChangeNotification(BKSyncLogReader.this);
- readAheadReader.start(logSegments.getValue());
- return BoxedUnit.UNIT;
- }
- });
- }
-
- @VisibleForTesting
- ReadAheadEntryReader getReadAheadReader() {
- return readAheadReader;
- }
-
- @VisibleForTesting
- BKLogReadHandler getReadHandler() {
- return readHandler;
- }
-
- private Entry.Reader readNextEntry(boolean nonBlocking) throws IOException {
- Entry.Reader entry = null;
- if (nonBlocking) {
- return readAheadReader.getNextReadAheadEntry(0L, TimeUnit.MILLISECONDS);
- } else {
- while (!readAheadReader.isReadAheadCaughtUp()
- && null == readerException.get()
- && null == entry) {
- entry = readAheadReader.getNextReadAheadEntry(maxReadAheadWaitTime, TimeUnit.MILLISECONDS);
- }
- if (null != entry) {
- return entry;
- }
- // reader is caught up
- if (readAheadReader.isReadAheadCaughtUp()
- && null == readerException.get()) {
- entry = readAheadReader.getNextReadAheadEntry(maxReadAheadWaitTime, TimeUnit.MILLISECONDS);
- }
- return entry;
- }
- }
-
- private void markReaderAsIdle() throws IdleReaderException {
- idleReaderError.inc();
- IdleReaderException ire = new IdleReaderException("Sync reader on stream "
- + readHandler.getFullyQualifiedName()
- + " is idle for more than " + idleErrorThresholdMillis + " ms");
- readerException.compareAndSet(null, ire);
- throw ire;
- }
-
- @Override
- public synchronized LogRecordWithDLSN readNext(boolean nonBlocking)
- throws IOException {
- if (null != readerException.get()) {
- throw readerException.get();
- }
- LogRecordWithDLSN record = doReadNext(nonBlocking);
- // no record is returned, check if the reader becomes idle
- if (null == record && shouldCheckIdleReader) {
- if (readAheadReader.getNumCachedEntries() <= 0 &&
- readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS)) {
- markReaderAsIdle();
- }
- }
- return record;
- }
-
- private LogRecordWithDLSN doReadNext(boolean nonBlocking) throws IOException {
- LogRecordWithDLSN record = null;
-
- do {
- // fetch one record until we don't find any entry available in the readahead cache
- while (null == record) {
- if (null == currentEntry) {
- currentEntry = readNextEntry(nonBlocking);
- if (null == currentEntry) {
- return null;
- }
- }
- record = currentEntry.nextRecord();
- if (null == record) {
- currentEntry = null;
- }
- }
-
- // check if we reached the end of stream
- if (record.isEndOfStream()) {
- EndOfStreamException eos = new EndOfStreamException("End of Stream Reached for "
- + readHandler.getFullyQualifiedName());
- readerException.compareAndSet(null, eos);
- throw eos;
- }
- // skip control records
- if (record.isControl()) {
- record = null;
- continue;
- }
- if (!positioned) {
- if (record.getTransactionId() < startTransactionId.get()) {
- record = null;
- continue;
- } else {
- positioned = true;
- break;
- }
- } else {
- break;
- }
- } while (true);
- return record;
- }
-
- @Override
- public synchronized List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords)
- throws IOException {
- LinkedList<LogRecordWithDLSN> retList =
- new LinkedList<LogRecordWithDLSN>();
-
- int numRead = 0;
- LogRecordWithDLSN record = readNext(nonBlocking);
- while ((null != record)) {
- retList.add(record);
- numRead++;
- if (numRead >= numLogRecords) {
- break;
- }
- record = readNext(nonBlocking);
- }
- return retList;
- }
-
- @Override
- public Future<Void> asyncClose() {
- Promise<Void> closePromise;
- synchronized (this) {
- if (null != closeFuture) {
- return closeFuture;
- }
- closeFuture = closePromise = new Promise<Void>();
- }
- readHandler.unregisterListener(readAheadReader);
- readAheadReader.removeStateChangeNotification(this);
- Utils.closeSequence(bkdlm.getScheduler(), true,
- readAheadReader,
- readHandler
- ).proxyTo(closePromise);
- return closePromise;
- }
-
- @Override
- public void close() throws IOException {
- FutureUtils.result(asyncClose());
- }
-
- //
- // Notification From ReadHandler
- //
-
- @Override
- public void notifyOnError(Throwable cause) {
- if (cause instanceof IOException) {
- readerException.compareAndSet(null, (IOException) cause);
- } else {
- readerException.compareAndSet(null, new IOException(cause));
- }
- }
-
- @Override
- public void notifyOnOperationComplete() {
- // no-op
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogWriter.java
deleted file mode 100644
index b638020..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogWriter.java
+++ /dev/null
@@ -1,113 +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.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.util.FutureUtils;
-
-import java.io.IOException;
-import java.util.List;
-
-class BKSyncLogWriter extends BKAbstractLogWriter implements LogWriter {
-
- public BKSyncLogWriter(DistributedLogConfiguration conf,
- DynamicDistributedLogConfiguration dynConf,
- BKDistributedLogManager bkdlm) {
- super(conf, dynConf, bkdlm);
- }
- /**
- * Write log records to the stream.
- *
- * @param record operation
- */
- @Override
- public void write(LogRecord record) throws IOException {
- getLedgerWriter(record.getTransactionId(), false).write(record);
- }
-
- /**
- * Write edits logs operation to the stream.
- *
- * @param records list of records
- */
- @Override
- @Deprecated
- public int writeBulk(List<LogRecord> records) throws IOException {
- return getLedgerWriter(records.get(0).getTransactionId(), false).writeBulk(records);
- }
-
- /**
- * Flushes all the data up to this point,
- * adds the end of stream marker and marks the stream
- * as read-only in the metadata. No appends to the
- * stream will be allowed after this point
- */
- @Override
- public void markEndOfStream() throws IOException {
- FutureUtils.result(getLedgerWriter(DistributedLogConstants.MAX_TXID, true).markEndOfStream());
- closeAndComplete();
- }
-
- /**
- * All data that has been written to the stream so far will be flushed.
- * New data can be still written to the stream while flush is ongoing.
- */
- @Override
- public long setReadyToFlush() throws IOException {
- checkClosedOrInError("setReadyToFlush");
- long highestTransactionId = 0;
- BKLogSegmentWriter writer = getCachedLogWriter();
- if (null != writer) {
- highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.flush()));
- }
- return highestTransactionId;
- }
-
- /**
- * Commit data that is already flushed.
- * <p/>
- * This API is optional as the writer implements a policy for automatically syncing
- * the log records in the buffer. The buffered edits can be flushed when the buffer
- * becomes full or a certain period of time is elapsed.
- */
- @Override
- public long flushAndSync() throws IOException {
- checkClosedOrInError("flushAndSync");
-
- LOG.debug("FlushAndSync Started");
- long highestTransactionId = 0;
- BKLogSegmentWriter writer = getCachedLogWriter();
- if (null != writer) {
- highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.commit()));
- LOG.debug("FlushAndSync Completed");
- } else {
- LOG.debug("FlushAndSync Completed - Nothing to Flush");
- }
- return highestTransactionId;
- }
-
- /**
- * Close the stream without necessarily flushing immediately.
- * This may be called if the stream is in error such as after a
- * previous write or close threw an exception.
- */
- @Override
- public void abort() throws IOException {
- super.abort();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKTransmitPacket.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKTransmitPacket.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKTransmitPacket.java
deleted file mode 100644
index 4586602..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKTransmitPacket.java
+++ /dev/null
@@ -1,90 +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.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-
-import java.util.concurrent.TimeUnit;
-
-class BKTransmitPacket {
-
- private final EntryBuffer recordSet;
- private final long transmitTime;
- private final Promise<Integer> transmitComplete;
-
- BKTransmitPacket(EntryBuffer recordSet) {
- this.recordSet = recordSet;
- this.transmitTime = System.nanoTime();
- this.transmitComplete = new Promise<Integer>();
- }
-
- EntryBuffer getRecordSet() {
- return recordSet;
- }
-
- Promise<Integer> getTransmitFuture() {
- return transmitComplete;
- }
-
- /**
- * Complete the transmit with result code <code>transmitRc</code>.
- * <p>It would notify all the waiters that are waiting via {@link #awaitTransmitComplete(long, TimeUnit)}
- * or {@link #addTransmitCompleteListener(FutureEventListener)}.
- *
- * @param transmitResult
- * transmit result code.
- */
- public void notifyTransmitComplete(int transmitResult) {
- transmitComplete.setValue(transmitResult);
- }
-
- /**
- * Register a transmit complete listener.
- * <p>The listener will be triggered with transmit result when transmit completes.
- * The method should be non-blocking.
- *
- * @param transmitCompleteListener
- * listener on transmit completion
- * @see #awaitTransmitComplete(long, TimeUnit)
- */
- void addTransmitCompleteListener(FutureEventListener<Integer> transmitCompleteListener) {
- transmitComplete.addEventListener(transmitCompleteListener);
- }
-
- /**
- * Await for the transmit to be complete
- *
- * @param timeout
- * wait timeout
- * @param unit
- * wait timeout unit
- */
- int awaitTransmitComplete(long timeout, TimeUnit unit)
- throws Exception {
- return Await.result(transmitComplete,
- Duration.fromTimeUnit(timeout, unit));
- }
-
- public long getTransmitTime() {
- return transmitTime;
- }
-
-}
[39/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java
new file mode 100644
index 0000000..d7494de
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java
@@ -0,0 +1,383 @@
+/**
+ * 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 org.apache.distributedlog.client;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.LogRecordSet;
+import org.apache.distributedlog.LogRecordSetBuffer;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.service.DistributedLogClient;
+import com.twitter.finagle.IndividualRequestTimeoutException;
+import com.twitter.util.Await;
+import com.twitter.util.Future;
+import com.twitter.util.Promise;
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test {@link DistributedLogMultiStreamWriter}.
+ */
+public class TestDistributedLogMultiStreamWriter {
+
+ @Test(timeout = 20000, expected = IllegalArgumentException.class)
+ public void testBuildWithNullStreams() throws Exception {
+ DistributedLogMultiStreamWriter.newBuilder()
+ .build();
+ }
+
+ @Test(timeout = 20000, expected = IllegalArgumentException.class)
+ public void testBuildWithEmptyStreamList() throws Exception {
+ DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.<String>newArrayList())
+ .build();
+ }
+
+ @Test(timeout = 20000, expected = NullPointerException.class)
+ public void testBuildWithNullClient() throws Exception {
+ DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .build();
+ }
+
+ @Test(timeout = 20000, expected = NullPointerException.class)
+ public void testBuildWithNullCodec() throws Exception {
+ DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(mock(DistributedLogClient.class))
+ .compressionCodec(null)
+ .build();
+ }
+
+ @Test(timeout = 20000, expected = IllegalArgumentException.class)
+ public void testBuildWithInvalidSpeculativeSettings1()
+ throws Exception {
+ DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(mock(DistributedLogClient.class))
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(-1)
+ .build();
+ }
+
+ @Test(timeout = 20000, expected = IllegalArgumentException.class)
+ public void testBuildWithInvalidSpeculativeSettings2()
+ throws Exception {
+ DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(mock(DistributedLogClient.class))
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(10)
+ .maxSpeculativeTimeoutMs(5)
+ .build();
+ }
+
+ @Test(timeout = 20000, expected = IllegalArgumentException.class)
+ public void testBuildWithInvalidSpeculativeSettings3()
+ throws Exception {
+ DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(mock(DistributedLogClient.class))
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(10)
+ .maxSpeculativeTimeoutMs(20)
+ .speculativeBackoffMultiplier(-1)
+ .build();
+ }
+
+ @Test(timeout = 20000, expected = IllegalArgumentException.class)
+ public void testBuildWithInvalidSpeculativeSettings4()
+ throws Exception {
+ DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(mock(DistributedLogClient.class))
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(10)
+ .maxSpeculativeTimeoutMs(20)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(10)
+ .build();
+ }
+
+ @Test(timeout = 20000)
+ public void testBuildMultiStreamWriter()
+ throws Exception {
+ DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(mock(DistributedLogClient.class))
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(10)
+ .maxSpeculativeTimeoutMs(20)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(50)
+ .build();
+ assertTrue(true);
+ }
+
+ @Test(timeout = 20000)
+ public void testBuildWithPeriodicalFlushEnabled() throws Exception {
+ ScheduledExecutorService executorService = mock(ScheduledExecutorService.class);
+ DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(mock(DistributedLogClient.class))
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(10)
+ .maxSpeculativeTimeoutMs(20)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(50)
+ .flushIntervalMs(1000)
+ .scheduler(executorService)
+ .build();
+ verify(executorService, times(1)).scheduleAtFixedRate(writer, 1000000, 1000000, TimeUnit.MICROSECONDS);
+ }
+
+ @Test(timeout = 20000)
+ public void testBuildWithPeriodicalFlushDisabled() throws Exception {
+ ScheduledExecutorService executorService = mock(ScheduledExecutorService.class);
+ DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(mock(DistributedLogClient.class))
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(10)
+ .maxSpeculativeTimeoutMs(20)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(50)
+ .flushIntervalMs(0)
+ .scheduler(executorService)
+ .build();
+ verify(executorService, times(0)).scheduleAtFixedRate(writer, 1000, 1000, TimeUnit.MILLISECONDS);
+ writer.close();
+ }
+
+ @Test(timeout = 20000)
+ public void testFlushWhenBufferIsFull() throws Exception {
+ DistributedLogClient client = mock(DistributedLogClient.class);
+ when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
+ .thenReturn(Future.value(new DLSN(1L, 1L, 999L)));
+
+ ScheduledExecutorService executorService =
+ Executors.newSingleThreadScheduledExecutor();
+ DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(client)
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(100000)
+ .maxSpeculativeTimeoutMs(200000)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(500000)
+ .flushIntervalMs(0)
+ .bufferSize(0)
+ .scheduler(executorService)
+ .build();
+
+ ByteBuffer buffer = ByteBuffer.wrap("test".getBytes(UTF_8));
+ writer.write(buffer);
+
+ verify(client, times(1)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+
+ writer.close();
+ }
+
+ @Test(timeout = 20000)
+ public void testFlushWhenExceedMaxLogRecordSetSize()
+ throws Exception {
+ DistributedLogClient client = mock(DistributedLogClient.class);
+ when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
+ .thenReturn(Future.value(new DLSN(1L, 1L, 999L)));
+ ScheduledExecutorService executorService =
+ Executors.newSingleThreadScheduledExecutor();
+ DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(client)
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(100000)
+ .maxSpeculativeTimeoutMs(200000)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(500000)
+ .flushIntervalMs(0)
+ .bufferSize(Integer.MAX_VALUE)
+ .scheduler(executorService)
+ .build();
+
+ byte[] data = new byte[LogRecord.MAX_LOGRECORD_SIZE - 3 * 100];
+ ByteBuffer buffer1 = ByteBuffer.wrap(data);
+ writer.write(buffer1);
+ verify(client, times(0)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+ LogRecordSet.Writer recordSetWriter1 = writer.getLogRecordSetWriter();
+ assertEquals(1, recordSetWriter1.getNumRecords());
+ assertEquals(LogRecordSet.HEADER_LEN + 4 + data.length, recordSetWriter1.getNumBytes());
+
+ ByteBuffer buffer2 = ByteBuffer.wrap(data);
+ writer.write(buffer2);
+ verify(client, times(1)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+ LogRecordSet.Writer recordSetWriter2 = writer.getLogRecordSetWriter();
+ assertEquals(1, recordSetWriter2.getNumRecords());
+ assertEquals(LogRecordSet.HEADER_LEN + 4 + data.length, recordSetWriter2.getNumBytes());
+ assertTrue(recordSetWriter1 != recordSetWriter2);
+
+ writer.close();
+ }
+
+ @Test(timeout = 20000)
+ public void testWriteTooLargeRecord() throws Exception {
+ DistributedLogClient client = mock(DistributedLogClient.class);
+ DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(client)
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(100000)
+ .maxSpeculativeTimeoutMs(200000)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(5000000)
+ .flushIntervalMs(0)
+ .bufferSize(0)
+ .build();
+
+ byte[] data = new byte[LogRecord.MAX_LOGRECORD_SIZE + 10];
+ ByteBuffer buffer = ByteBuffer.wrap(data);
+ Future<DLSN> writeFuture = writer.write(buffer);
+ assertTrue(writeFuture.isDefined());
+ try {
+ Await.result(writeFuture);
+ fail("Should fail on writing too long record");
+ } catch (LogRecordTooLongException lrtle) {
+ // expected
+ }
+ writer.close();
+ }
+
+ @Test(timeout = 20000)
+ public void testSpeculativeWrite() throws Exception {
+ DistributedLogClient client = mock(DistributedLogClient.class);
+ DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(client)
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(10)
+ .maxSpeculativeTimeoutMs(20)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(5000000)
+ .flushIntervalMs(0)
+ .bufferSize(0)
+ .build();
+
+ final String secondStream = writer.getStream(1);
+
+ final DLSN dlsn = new DLSN(99L, 88L, 0L);
+
+ Mockito.doAnswer(new Answer() {
+ @Override
+ public Object answer(InvocationOnMock invocation) throws Throwable {
+ Object[] arguments = invocation.getArguments();
+ String stream = (String) arguments[0];
+ if (stream.equals(secondStream)) {
+ return Future.value(dlsn);
+ } else {
+ return new Promise<DLSN>();
+ }
+ }
+ }).when(client).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+
+ byte[] data = "test-test".getBytes(UTF_8);
+ ByteBuffer buffer = ByteBuffer.wrap(data);
+ Future<DLSN> writeFuture = writer.write(buffer);
+ DLSN writeDLSN = Await.result(writeFuture);
+ assertEquals(dlsn, writeDLSN);
+ writer.close();
+ }
+
+ @Test(timeout = 20000)
+ public void testPeriodicalFlush() throws Exception {
+ DistributedLogClient client = mock(DistributedLogClient.class);
+ DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(client)
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(10)
+ .maxSpeculativeTimeoutMs(20)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(5000000)
+ .flushIntervalMs(10)
+ .bufferSize(Integer.MAX_VALUE)
+ .build();
+
+ final DLSN dlsn = new DLSN(99L, 88L, 0L);
+
+ Mockito.doAnswer(new Answer() {
+ @Override
+ public Object answer(InvocationOnMock invocation) throws Throwable {
+ return Future.value(dlsn);
+ }
+ }).when(client).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+
+ byte[] data = "test-test".getBytes(UTF_8);
+ ByteBuffer buffer = ByteBuffer.wrap(data);
+ Future<DLSN> writeFuture = writer.write(buffer);
+ DLSN writeDLSN = Await.result(writeFuture);
+ assertEquals(dlsn, writeDLSN);
+ writer.close();
+ }
+
+ @Test(timeout = 20000)
+ public void testFailRequestAfterRetriedAllStreams() throws Exception {
+ DistributedLogClient client = mock(DistributedLogClient.class);
+ when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
+ .thenReturn(new Promise<DLSN>());
+ DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+ .streams(Lists.newArrayList("stream1", "stream2"))
+ .client(client)
+ .compressionCodec(CompressionCodec.Type.LZ4)
+ .firstSpeculativeTimeoutMs(10)
+ .maxSpeculativeTimeoutMs(20)
+ .speculativeBackoffMultiplier(2)
+ .requestTimeoutMs(5000000)
+ .flushIntervalMs(10)
+ .bufferSize(Integer.MAX_VALUE)
+ .build();
+
+ byte[] data = "test-test".getBytes(UTF_8);
+ ByteBuffer buffer = ByteBuffer.wrap(data);
+ Future<DLSN> writeFuture = writer.write(buffer);
+ try {
+ Await.result(writeFuture);
+ fail("Should fail the request after retries all streams");
+ } catch (IndividualRequestTimeoutException e) {
+ long timeoutMs = e.timeout().inMilliseconds();
+ assertTrue(timeoutMs >= (10 + 20) && timeoutMs < 5000000);
+ }
+ writer.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java
new file mode 100644
index 0000000..86d1c11
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java
@@ -0,0 +1,207 @@
+/**
+ * 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 org.apache.distributedlog.client.ownership;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.distributedlog.client.ClientConfig;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.Map;
+import java.util.Set;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test Case for Ownership Cache.
+ */
+public class TestOwnershipCache {
+
+ @Rule
+ public TestName runtime = new TestName();
+
+ private static OwnershipCache createOwnershipCache() {
+ ClientConfig clientConfig = new ClientConfig();
+ return new OwnershipCache(clientConfig, null,
+ NullStatsReceiver.get(), NullStatsReceiver.get());
+ }
+
+ private static SocketAddress createSocketAddress(int port) {
+ return new InetSocketAddress("127.0.0.1", port);
+ }
+
+ @Test(timeout = 60000)
+ public void testUpdateOwner() {
+ OwnershipCache cache = createOwnershipCache();
+ SocketAddress addr = createSocketAddress(1000);
+ String stream = runtime.getMethodName();
+
+ assertTrue("Should successfully update owner if no owner exists before",
+ cache.updateOwner(stream, addr));
+ assertEquals("Owner should be " + addr + " for stream " + stream,
+ addr, cache.getOwner(stream));
+ assertTrue("Should successfully update owner if old owner is same",
+ cache.updateOwner(stream, addr));
+ assertEquals("Owner should be " + addr + " for stream " + stream,
+ addr, cache.getOwner(stream));
+ }
+
+ @Test(timeout = 60000)
+ public void testRemoveOwnerFromStream() {
+ OwnershipCache cache = createOwnershipCache();
+ int initialPort = 2000;
+ int numProxies = 2;
+ int numStreamsPerProxy = 2;
+ for (int i = 0; i < numProxies; i++) {
+ SocketAddress addr = createSocketAddress(initialPort + i);
+ for (int j = 0; j < numStreamsPerProxy; j++) {
+ String stream = runtime.getMethodName() + "_" + i + "_" + j;
+ cache.updateOwner(stream, addr);
+ }
+ }
+ Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
+ assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+ numProxies * numStreamsPerProxy, ownershipMap.size());
+ Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
+ assertEquals("There should be " + numProxies + " proxies cached",
+ numProxies, ownershipDistribution.size());
+
+ String stream = runtime.getMethodName() + "_0_0";
+ SocketAddress owner = createSocketAddress(initialPort);
+
+ // remove non-existent mapping won't change anything
+ SocketAddress nonExistentAddr = createSocketAddress(initialPort + 999);
+ cache.removeOwnerFromStream(stream, nonExistentAddr, "remove-non-existent-addr");
+ assertEquals("Owner " + owner + " should not be removed",
+ owner, cache.getOwner(stream));
+ ownershipMap = cache.getStreamOwnerMapping();
+ assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+ numProxies * numStreamsPerProxy, ownershipMap.size());
+
+ // remove existent mapping should remove ownership mapping
+ cache.removeOwnerFromStream(stream, owner, "remove-owner");
+ assertNull("Owner " + owner + " should be removed", cache.getOwner(stream));
+ ownershipMap = cache.getStreamOwnerMapping();
+ assertEquals("There should be " + (numProxies * numStreamsPerProxy - 1) + " entries left in cache",
+ numProxies * numStreamsPerProxy - 1, ownershipMap.size());
+ ownershipDistribution = cache.getStreamOwnershipDistribution();
+ assertEquals("There should still be " + numProxies + " proxies cached",
+ numProxies, ownershipDistribution.size());
+ Set<String> ownedStreams = ownershipDistribution.get(owner);
+ assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned for " + owner,
+ numStreamsPerProxy - 1, ownedStreams.size());
+ assertFalse("Stream " + stream + " should not be owned by " + owner,
+ ownedStreams.contains(stream));
+ }
+
+ @Test(timeout = 60000)
+ public void testRemoveAllStreamsFromOwner() {
+ OwnershipCache cache = createOwnershipCache();
+ int initialPort = 2000;
+ int numProxies = 2;
+ int numStreamsPerProxy = 2;
+ for (int i = 0; i < numProxies; i++) {
+ SocketAddress addr = createSocketAddress(initialPort + i);
+ for (int j = 0; j < numStreamsPerProxy; j++) {
+ String stream = runtime.getMethodName() + "_" + i + "_" + j;
+ cache.updateOwner(stream, addr);
+ }
+ }
+ Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
+ assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+ numProxies * numStreamsPerProxy, ownershipMap.size());
+ Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
+ assertEquals("There should be " + numProxies + " proxies cached",
+ numProxies, ownershipDistribution.size());
+
+ SocketAddress owner = createSocketAddress(initialPort);
+
+ // remove non-existent host won't change anything
+ SocketAddress nonExistentAddr = createSocketAddress(initialPort + 999);
+ cache.removeAllStreamsFromOwner(nonExistentAddr);
+ ownershipMap = cache.getStreamOwnerMapping();
+ assertEquals("There should still be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+ numProxies * numStreamsPerProxy, ownershipMap.size());
+ ownershipDistribution = cache.getStreamOwnershipDistribution();
+ assertEquals("There should still be " + numProxies + " proxies cached",
+ numProxies, ownershipDistribution.size());
+
+ // remove existent host should remove ownership mapping
+ cache.removeAllStreamsFromOwner(owner);
+ ownershipMap = cache.getStreamOwnerMapping();
+ assertEquals("There should be " + ((numProxies - 1) * numStreamsPerProxy) + " entries left in cache",
+ (numProxies - 1) * numStreamsPerProxy, ownershipMap.size());
+ ownershipDistribution = cache.getStreamOwnershipDistribution();
+ assertEquals("There should be " + (numProxies - 1) + " proxies cached",
+ numProxies - 1, ownershipDistribution.size());
+ assertFalse("Host " + owner + " should not be cached",
+ ownershipDistribution.containsKey(owner));
+ }
+
+ @Test(timeout = 60000)
+ public void testReplaceOwner() {
+ OwnershipCache cache = createOwnershipCache();
+ int initialPort = 2000;
+ int numProxies = 2;
+ int numStreamsPerProxy = 2;
+ for (int i = 0; i < numProxies; i++) {
+ SocketAddress addr = createSocketAddress(initialPort + i);
+ for (int j = 0; j < numStreamsPerProxy; j++) {
+ String stream = runtime.getMethodName() + "_" + i + "_" + j;
+ cache.updateOwner(stream, addr);
+ }
+ }
+ Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
+ assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+ numProxies * numStreamsPerProxy, ownershipMap.size());
+ Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
+ assertEquals("There should be " + numProxies + " proxies cached",
+ numProxies, ownershipDistribution.size());
+
+ String stream = runtime.getMethodName() + "_0_0";
+ SocketAddress oldOwner = createSocketAddress(initialPort);
+ SocketAddress newOwner = createSocketAddress(initialPort + 999);
+
+ cache.updateOwner(stream, newOwner);
+ assertEquals("Owner of " + stream + " should be changed from " + oldOwner + " to " + newOwner,
+ newOwner, cache.getOwner(stream));
+ ownershipMap = cache.getStreamOwnerMapping();
+ assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+ numProxies * numStreamsPerProxy, ownershipMap.size());
+ assertEquals("Owner of " + stream + " should be " + newOwner,
+ newOwner, ownershipMap.get(stream));
+ ownershipDistribution = cache.getStreamOwnershipDistribution();
+ assertEquals("There should be " + (numProxies + 1) + " proxies cached",
+ numProxies + 1, ownershipDistribution.size());
+ Set<String> oldOwnedStreams = ownershipDistribution.get(oldOwner);
+ assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned by " + oldOwner,
+ numStreamsPerProxy - 1, oldOwnedStreams.size());
+ assertFalse("Stream " + stream + " should not be owned by " + oldOwner,
+ oldOwnedStreams.contains(stream));
+ Set<String> newOwnedStreams = ownershipDistribution.get(newOwner);
+ assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned by " + newOwner,
+ 1, newOwnedStreams.size());
+ assertTrue("Stream " + stream + " should be owned by " + newOwner,
+ newOwnedStreams.contains(stream));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java
new file mode 100644
index 0000000..8ef33bd
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java
@@ -0,0 +1,144 @@
+/**
+ * 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 org.apache.distributedlog.client.proxy;
+
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.ClientInfo;
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import org.apache.distributedlog.thrift.service.HeartbeatOptions;
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import org.apache.distributedlog.thrift.service.WriteContext;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import com.twitter.util.Future;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Mock DistributedLog Related Services.
+ */
+public class MockDistributedLogServices {
+
+ /**
+ * Mock basic service.
+ */
+ static class MockBasicService implements DistributedLogService.ServiceIface {
+
+ @Override
+ public Future<ServerInfo> handshake() {
+ return Future.value(new ServerInfo());
+ }
+
+ @Override
+ public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
+ return Future.value(new ServerInfo());
+ }
+
+ @Override
+ public Future<WriteResponse> heartbeat(String stream, WriteContext ctx) {
+ return Future.value(new WriteResponse());
+ }
+
+ @Override
+ public Future<WriteResponse> heartbeatWithOptions(String stream,
+ WriteContext ctx,
+ HeartbeatOptions options) {
+ return Future.value(new WriteResponse());
+ }
+
+ @Override
+ public Future<WriteResponse> write(String stream,
+ ByteBuffer data) {
+ return Future.value(new WriteResponse());
+ }
+
+ @Override
+ public Future<WriteResponse> writeWithContext(String stream,
+ ByteBuffer data,
+ WriteContext ctx) {
+ return Future.value(new WriteResponse());
+ }
+
+ @Override
+ public Future<BulkWriteResponse> writeBulkWithContext(String stream,
+ List<ByteBuffer> data,
+ WriteContext ctx) {
+ return Future.value(new BulkWriteResponse());
+ }
+
+ @Override
+ public Future<WriteResponse> truncate(String stream,
+ String dlsn,
+ WriteContext ctx) {
+ return Future.value(new WriteResponse());
+ }
+
+ @Override
+ public Future<WriteResponse> release(String stream,
+ WriteContext ctx) {
+ return Future.value(new WriteResponse());
+ }
+
+ @Override
+ public Future<WriteResponse> create(String stream, WriteContext ctx) {
+ return Future.value(new WriteResponse());
+ }
+
+ @Override
+ public Future<WriteResponse> delete(String stream,
+ WriteContext ctx) {
+ return Future.value(new WriteResponse());
+ }
+
+ @Override
+ public Future<WriteResponse> getOwner(String stream, WriteContext ctx) {
+ return Future.value(new WriteResponse());
+ }
+
+ @Override
+ public Future<Void> setAcceptNewStream(boolean enabled) {
+ return Future.value(null);
+ }
+ }
+
+ /**
+ * Mock server info service.
+ */
+ public static class MockServerInfoService extends MockBasicService {
+
+ protected ServerInfo serverInfo;
+
+ public MockServerInfoService() {
+ serverInfo = new ServerInfo();
+ }
+
+ public void updateServerInfo(ServerInfo serverInfo) {
+ this.serverInfo = serverInfo;
+ }
+
+ @Override
+ public Future<ServerInfo> handshake() {
+ return Future.value(serverInfo);
+ }
+
+ @Override
+ public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
+ return Future.value(serverInfo);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java
new file mode 100644
index 0000000..e38c2ed
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java
@@ -0,0 +1,49 @@
+/**
+ * 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 org.apache.distributedlog.client.proxy;
+
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import java.net.SocketAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Mock Proxy Client Builder.
+ */
+class MockProxyClientBuilder implements ProxyClient.Builder {
+
+ static class MockProxyClient extends ProxyClient {
+ MockProxyClient(SocketAddress address,
+ DistributedLogService.ServiceIface service) {
+ super(address, new MockThriftClient(), service);
+ }
+ }
+
+ private final ConcurrentMap<SocketAddress, MockProxyClient> clients =
+ new ConcurrentHashMap<SocketAddress, MockProxyClient>();
+
+ public void provideProxyClient(SocketAddress address,
+ MockProxyClient proxyClient) {
+ clients.put(address, proxyClient);
+ }
+
+ @Override
+ public ProxyClient build(SocketAddress address) {
+ return clients.get(address);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java
new file mode 100644
index 0000000..ad1c878
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java
@@ -0,0 +1,32 @@
+/**
+ * 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 org.apache.distributedlog.client.proxy;
+
+import com.twitter.finagle.Service;
+import com.twitter.finagle.thrift.ThriftClientRequest;
+import com.twitter.util.Future;
+
+/**
+ * Mock Thrift Client.
+ */
+class MockThriftClient extends Service<ThriftClientRequest, byte[]> {
+ @Override
+ public Future<byte[]> apply(ThriftClientRequest request) {
+ return Future.value(request.message);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java
new file mode 100644
index 0000000..6d9a471
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java
@@ -0,0 +1,368 @@
+/**
+ * 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 org.apache.distributedlog.client.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.proxy.MockDistributedLogServices.MockBasicService;
+import org.apache.distributedlog.client.proxy.MockDistributedLogServices.MockServerInfoService;
+import org.apache.distributedlog.client.proxy.MockProxyClientBuilder.MockProxyClient;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.client.stats.ClientStats;
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.tuple.Pair;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test Proxy Client Manager.
+ */
+public class TestProxyClientManager {
+
+ @Rule
+ public TestName runtime = new TestName();
+
+ static class TestHostProvider implements HostProvider {
+
+ Set<SocketAddress> hosts = new HashSet<SocketAddress>();
+
+ synchronized void addHost(SocketAddress host) {
+ hosts.add(host);
+ }
+
+ @Override
+ public synchronized Set<SocketAddress> getHosts() {
+ return ImmutableSet.copyOf(hosts);
+ }
+
+ }
+
+ private static ProxyClientManager createProxyClientManager(ProxyClient.Builder builder,
+ long periodicHandshakeIntervalMs) {
+ HostProvider provider = new TestHostProvider();
+ return createProxyClientManager(builder, provider, periodicHandshakeIntervalMs);
+ }
+
+ private static ProxyClientManager createProxyClientManager(ProxyClient.Builder builder,
+ HostProvider hostProvider,
+ long periodicHandshakeIntervalMs) {
+ ClientConfig clientConfig = new ClientConfig();
+ clientConfig.setPeriodicHandshakeIntervalMs(periodicHandshakeIntervalMs);
+ clientConfig.setPeriodicOwnershipSyncIntervalMs(-1);
+ HashedWheelTimer dlTimer = new HashedWheelTimer(
+ new ThreadFactoryBuilder().setNameFormat("TestProxyClientManager-timer-%d").build(),
+ clientConfig.getRedirectBackoffStartMs(),
+ TimeUnit.MILLISECONDS);
+ return new ProxyClientManager(clientConfig, builder, dlTimer, hostProvider,
+ new ClientStats(NullStatsReceiver.get(), false, new DefaultRegionResolver()));
+ }
+
+ private static SocketAddress createSocketAddress(int port) {
+ return new InetSocketAddress("127.0.0.1", port);
+ }
+
+ private static MockProxyClient createMockProxyClient(SocketAddress address) {
+ return new MockProxyClient(address, new MockBasicService());
+ }
+
+ private static Pair<MockProxyClient, MockServerInfoService> createMockProxyClient(
+ SocketAddress address, ServerInfo serverInfo) {
+ MockServerInfoService service = new MockServerInfoService();
+ MockProxyClient proxyClient = new MockProxyClient(address, service);
+ service.updateServerInfo(serverInfo);
+ return Pair.of(proxyClient, service);
+ }
+
+ @Test(timeout = 60000)
+ public void testBasicCreateRemove() throws Exception {
+ SocketAddress address = createSocketAddress(1000);
+ MockProxyClientBuilder builder = new MockProxyClientBuilder();
+ MockProxyClient mockProxyClient = createMockProxyClient(address);
+ builder.provideProxyClient(address, mockProxyClient);
+
+ ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+ assertEquals("There should be no clients in the manager",
+ 0, clientManager.getNumProxies());
+ ProxyClient proxyClient = clientManager.createClient(address);
+ assertEquals("Create client should build the proxy client",
+ 1, clientManager.getNumProxies());
+ assertTrue("The client returned should be the same client that builder built",
+ mockProxyClient == proxyClient);
+ }
+
+ @Test(timeout = 60000)
+ public void testGetShouldCreateClient() throws Exception {
+ SocketAddress address = createSocketAddress(2000);
+ MockProxyClientBuilder builder = new MockProxyClientBuilder();
+ MockProxyClient mockProxyClient = createMockProxyClient(address);
+ builder.provideProxyClient(address, mockProxyClient);
+
+ ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+ assertEquals("There should be no clients in the manager",
+ 0, clientManager.getNumProxies());
+ ProxyClient proxyClient = clientManager.getClient(address);
+ assertEquals("Get client should build the proxy client",
+ 1, clientManager.getNumProxies());
+ assertTrue("The client returned should be the same client that builder built",
+ mockProxyClient == proxyClient);
+ }
+
+ @Test(timeout = 60000)
+ public void testConditionalRemoveClient() throws Exception {
+ SocketAddress address = createSocketAddress(3000);
+ MockProxyClientBuilder builder = new MockProxyClientBuilder();
+ MockProxyClient mockProxyClient = createMockProxyClient(address);
+ MockProxyClient anotherMockProxyClient = createMockProxyClient(address);
+ builder.provideProxyClient(address, mockProxyClient);
+
+ ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+ assertEquals("There should be no clients in the manager",
+ 0, clientManager.getNumProxies());
+ clientManager.createClient(address);
+ assertEquals("Create client should build the proxy client",
+ 1, clientManager.getNumProxies());
+ clientManager.removeClient(address, anotherMockProxyClient);
+ assertEquals("Conditional remove should not remove proxy client",
+ 1, clientManager.getNumProxies());
+ clientManager.removeClient(address, mockProxyClient);
+ assertEquals("Conditional remove should remove proxy client",
+ 0, clientManager.getNumProxies());
+ }
+
+ @Test(timeout = 60000)
+ public void testRemoveClient() throws Exception {
+ SocketAddress address = createSocketAddress(3000);
+ MockProxyClientBuilder builder = new MockProxyClientBuilder();
+ MockProxyClient mockProxyClient = createMockProxyClient(address);
+ builder.provideProxyClient(address, mockProxyClient);
+
+ ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+ assertEquals("There should be no clients in the manager",
+ 0, clientManager.getNumProxies());
+ clientManager.createClient(address);
+ assertEquals("Create client should build the proxy client",
+ 1, clientManager.getNumProxies());
+ clientManager.removeClient(address);
+ assertEquals("Remove should remove proxy client",
+ 0, clientManager.getNumProxies());
+ }
+
+ @Test(timeout = 60000)
+ public void testCreateClientShouldHandshake() throws Exception {
+ SocketAddress address = createSocketAddress(3000);
+ MockProxyClientBuilder builder = new MockProxyClientBuilder();
+ ServerInfo serverInfo = new ServerInfo();
+ serverInfo.putToOwnerships(runtime.getMethodName() + "_stream",
+ runtime.getMethodName() + "_owner");
+ Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
+ createMockProxyClient(address, serverInfo);
+ builder.provideProxyClient(address, mockProxyClient.getLeft());
+
+ final AtomicReference<ServerInfo> resultHolder = new AtomicReference<ServerInfo>(null);
+ final CountDownLatch doneLatch = new CountDownLatch(1);
+ ProxyListener listener = new ProxyListener() {
+ @Override
+ public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
+ resultHolder.set(serverInfo);
+ doneLatch.countDown();
+ }
+ @Override
+ public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
+ }
+ };
+
+ ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+ clientManager.registerProxyListener(listener);
+ assertEquals("There should be no clients in the manager",
+ 0, clientManager.getNumProxies());
+ clientManager.createClient(address);
+ assertEquals("Create client should build the proxy client",
+ 1, clientManager.getNumProxies());
+
+ // When a client is created, it would handshake with that proxy
+ doneLatch.await();
+ assertEquals("Handshake should return server info",
+ serverInfo, resultHolder.get());
+ }
+
+ @Test(timeout = 60000)
+ public void testHandshake() throws Exception {
+ final int numHosts = 3;
+ final int numStreamsPerHost = 3;
+ final int initialPort = 4000;
+
+ MockProxyClientBuilder builder = new MockProxyClientBuilder();
+ Map<SocketAddress, ServerInfo> serverInfoMap =
+ new HashMap<SocketAddress, ServerInfo>();
+ for (int i = 0; i < numHosts; i++) {
+ SocketAddress address = createSocketAddress(initialPort + i);
+ ServerInfo serverInfo = new ServerInfo();
+ for (int j = 0; j < numStreamsPerHost; j++) {
+ serverInfo.putToOwnerships(runtime.getMethodName() + "_stream_" + j,
+ address.toString());
+ }
+ Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
+ createMockProxyClient(address, serverInfo);
+ builder.provideProxyClient(address, mockProxyClient.getLeft());
+ serverInfoMap.put(address, serverInfo);
+ }
+
+ final Map<SocketAddress, ServerInfo> results = new HashMap<SocketAddress, ServerInfo>();
+ final CountDownLatch doneLatch = new CountDownLatch(2 * numHosts);
+ ProxyListener listener = new ProxyListener() {
+ @Override
+ public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
+ synchronized (results) {
+ results.put(address, serverInfo);
+ }
+ doneLatch.countDown();
+ }
+
+ @Override
+ public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
+ }
+ };
+
+ TestHostProvider rs = new TestHostProvider();
+ ProxyClientManager clientManager = createProxyClientManager(builder, rs, 0L);
+ clientManager.registerProxyListener(listener);
+ assertEquals("There should be no clients in the manager",
+ 0, clientManager.getNumProxies());
+ for (int i = 0; i < numHosts; i++) {
+ rs.addHost(createSocketAddress(initialPort + i));
+ }
+ // handshake would handshake with 3 hosts again
+ clientManager.handshake();
+ doneLatch.await();
+ assertEquals("Handshake should return server info",
+ numHosts, results.size());
+ assertTrue("Handshake should get all server infos",
+ Maps.difference(serverInfoMap, results).areEqual());
+ }
+
+ @Test(timeout = 60000)
+ public void testPeriodicHandshake() throws Exception {
+ final int numHosts = 3;
+ final int numStreamsPerHost = 3;
+ final int initialPort = 5000;
+
+ MockProxyClientBuilder builder = new MockProxyClientBuilder();
+ Map<SocketAddress, ServerInfo> serverInfoMap =
+ new HashMap<SocketAddress, ServerInfo>();
+ Map<SocketAddress, MockServerInfoService> mockServiceMap =
+ new HashMap<SocketAddress, MockServerInfoService>();
+ final Map<SocketAddress, CountDownLatch> hostDoneLatches =
+ new HashMap<SocketAddress, CountDownLatch>();
+ for (int i = 0; i < numHosts; i++) {
+ SocketAddress address = createSocketAddress(initialPort + i);
+ ServerInfo serverInfo = new ServerInfo();
+ for (int j = 0; j < numStreamsPerHost; j++) {
+ serverInfo.putToOwnerships(runtime.getMethodName() + "_stream_" + j,
+ address.toString());
+ }
+ Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
+ createMockProxyClient(address, serverInfo);
+ builder.provideProxyClient(address, mockProxyClient.getLeft());
+ serverInfoMap.put(address, serverInfo);
+ mockServiceMap.put(address, mockProxyClient.getRight());
+ hostDoneLatches.put(address, new CountDownLatch(2));
+ }
+
+ final Map<SocketAddress, ServerInfo> results = new HashMap<SocketAddress, ServerInfo>();
+ final CountDownLatch doneLatch = new CountDownLatch(numHosts);
+ ProxyListener listener = new ProxyListener() {
+ @Override
+ public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
+ synchronized (results) {
+ results.put(address, serverInfo);
+ CountDownLatch latch = hostDoneLatches.get(address);
+ if (null != latch) {
+ latch.countDown();
+ }
+ }
+ doneLatch.countDown();
+ }
+
+ @Override
+ public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
+ }
+ };
+
+ TestHostProvider rs = new TestHostProvider();
+ ProxyClientManager clientManager = createProxyClientManager(builder, rs, 50L);
+ clientManager.setPeriodicHandshakeEnabled(false);
+ clientManager.registerProxyListener(listener);
+
+ assertEquals("There should be no clients in the manager",
+ 0, clientManager.getNumProxies());
+ for (int i = 0; i < numHosts; i++) {
+ SocketAddress address = createSocketAddress(initialPort + i);
+ rs.addHost(address);
+ clientManager.createClient(address);
+ }
+
+ // make sure the first 3 handshakes going through
+ doneLatch.await();
+
+ assertEquals("Handshake should return server info",
+ numHosts, results.size());
+ assertTrue("Handshake should get all server infos",
+ Maps.difference(serverInfoMap, results).areEqual());
+
+ // update server info
+ for (int i = 0; i < numHosts; i++) {
+ SocketAddress address = createSocketAddress(initialPort + i);
+ ServerInfo serverInfo = new ServerInfo();
+ for (int j = 0; j < numStreamsPerHost; j++) {
+ serverInfo.putToOwnerships(runtime.getMethodName() + "_new_stream_" + j,
+ address.toString());
+ }
+ MockServerInfoService service = mockServiceMap.get(address);
+ serverInfoMap.put(address, serverInfo);
+ service.updateServerInfo(serverInfo);
+ }
+
+ clientManager.setPeriodicHandshakeEnabled(true);
+ for (int i = 0; i < numHosts; i++) {
+ SocketAddress address = createSocketAddress(initialPort + i);
+ CountDownLatch latch = hostDoneLatches.get(address);
+ latch.await();
+ }
+
+ assertTrue("Periodic handshake should update all server infos",
+ Maps.difference(serverInfoMap, results).areEqual());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java
new file mode 100644
index 0000000..f44cddd
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java
@@ -0,0 +1,417 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.service.DLSocketAddress;
+import com.twitter.finagle.Address;
+import com.twitter.finagle.Addresses;
+import com.twitter.finagle.ChannelWriteException;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ConsistentHashRoutingService}.
+ */
+public class TestConsistentHashRoutingService {
+
+ @Test(timeout = 60000)
+ public void testBlackoutHost() throws Exception {
+ TestName name = new TestName();
+ RoutingService routingService = ConsistentHashRoutingService.newBuilder()
+ .serverSet(new NameServerSet(name))
+ .resolveFromName(true)
+ .numReplicas(997)
+ .blackoutSeconds(2)
+ .build();
+
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", 3181);
+ Address address = Addresses.newInetAddress(inetAddress);
+ List<Address> addresses = new ArrayList<Address>(1);
+ addresses.add(address);
+ name.changeAddrs(addresses);
+
+ routingService.startService();
+
+ RoutingService.RoutingContext routingContext =
+ RoutingService.RoutingContext.of(new DefaultRegionResolver());
+
+ String streamName = "test-blackout-host";
+ assertEquals(inetAddress, routingService.getHost(streamName, routingContext));
+ routingService.removeHost(inetAddress, new ChannelWriteException(new IOException("test exception")));
+ try {
+ routingService.getHost(streamName, routingContext);
+ fail("Should fail to get host since no brokers are available");
+ } catch (NoBrokersAvailableException nbae) {
+ // expected
+ }
+
+ TimeUnit.SECONDS.sleep(3);
+ assertEquals(inetAddress, routingService.getHost(streamName, routingContext));
+
+ routingService.stopService();
+ }
+
+ @Test(timeout = 60000)
+ public void testPerformServerSetChangeOnName() throws Exception {
+ TestName name = new TestName();
+ ConsistentHashRoutingService routingService = (ConsistentHashRoutingService)
+ ConsistentHashRoutingService.newBuilder()
+ .serverSet(new NameServerSet(name))
+ .resolveFromName(true)
+ .numReplicas(997)
+ .build();
+
+ int basePort = 3180;
+ int numHosts = 4;
+ List<Address> addresses1 = Lists.newArrayListWithExpectedSize(4);
+ List<Address> addresses2 = Lists.newArrayListWithExpectedSize(4);
+ List<Address> addresses3 = Lists.newArrayListWithExpectedSize(4);
+
+ // fill up the addresses1
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+ Address address = Addresses.newInetAddress(inetAddress);
+ addresses1.add(address);
+ }
+ // fill up the addresses2 - overlap with addresses1
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 2 + i);
+ Address address = Addresses.newInetAddress(inetAddress);
+ addresses2.add(address);
+ }
+ // fill up the addresses3 - not overlap with addresses2
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
+ Address address = Addresses.newInetAddress(inetAddress);
+ addresses3.add(address);
+ }
+
+ final List<SocketAddress> leftAddresses = Lists.newArrayList();
+ final List<SocketAddress> joinAddresses = Lists.newArrayList();
+
+ RoutingService.RoutingListener routingListener = new RoutingService.RoutingListener() {
+ @Override
+ public void onServerLeft(SocketAddress address) {
+ synchronized (leftAddresses) {
+ leftAddresses.add(address);
+ leftAddresses.notifyAll();
+ }
+ }
+
+ @Override
+ public void onServerJoin(SocketAddress address) {
+ synchronized (joinAddresses) {
+ joinAddresses.add(address);
+ joinAddresses.notifyAll();
+ }
+ }
+ };
+
+ routingService.registerListener(routingListener);
+ name.changeAddrs(addresses1);
+
+ routingService.startService();
+
+ synchronized (joinAddresses) {
+ while (joinAddresses.size() < numHosts) {
+ joinAddresses.wait();
+ }
+ }
+
+ // validate 4 nodes joined
+ synchronized (joinAddresses) {
+ assertEquals(numHosts, joinAddresses.size());
+ }
+ synchronized (leftAddresses) {
+ assertEquals(0, leftAddresses.size());
+ }
+ assertEquals(numHosts, routingService.shardId2Address.size());
+ assertEquals(numHosts, routingService.address2ShardId.size());
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+ assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+ int shardId = routingService.address2ShardId.get(inetAddress);
+ SocketAddress sa = routingService.shardId2Address.get(shardId);
+ assertNotNull(sa);
+ assertEquals(inetAddress, sa);
+ }
+
+ // update addresses2 - 2 new hosts joined, 2 old hosts left
+ name.changeAddrs(addresses2);
+ synchronized (joinAddresses) {
+ while (joinAddresses.size() < numHosts + 2) {
+ joinAddresses.wait();
+ }
+ }
+ synchronized (leftAddresses) {
+ while (leftAddresses.size() < numHosts - 2) {
+ leftAddresses.wait();
+ }
+ }
+ assertEquals(numHosts, routingService.shardId2Address.size());
+ assertEquals(numHosts, routingService.address2ShardId.size());
+
+ // first 2 shards should leave
+ for (int i = 0; i < 2; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+ assertFalse(routingService.address2ShardId.containsKey(inetAddress));
+ }
+
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 2 + i);
+ assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+ int shardId = routingService.address2ShardId.get(inetAddress);
+ SocketAddress sa = routingService.shardId2Address.get(shardId);
+ assertNotNull(sa);
+ assertEquals(inetAddress, sa);
+ }
+
+ // update addresses3 - 2 new hosts joined, 2 old hosts left
+ name.changeAddrs(addresses3);
+ synchronized (joinAddresses) {
+ while (joinAddresses.size() < numHosts + 2 + numHosts) {
+ joinAddresses.wait();
+ }
+ }
+ synchronized (leftAddresses) {
+ while (leftAddresses.size() < numHosts - 2 + numHosts) {
+ leftAddresses.wait();
+ }
+ }
+ assertEquals(numHosts, routingService.shardId2Address.size());
+ assertEquals(numHosts, routingService.address2ShardId.size());
+
+ // first 6 shards should leave
+ for (int i = 0; i < 2 + numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+ assertFalse(routingService.address2ShardId.containsKey(inetAddress));
+ }
+ // new 4 shards should exist
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
+ assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+ int shardId = routingService.address2ShardId.get(inetAddress);
+ SocketAddress sa = routingService.shardId2Address.get(shardId);
+ assertNotNull(sa);
+ assertEquals(inetAddress, sa);
+ }
+
+ }
+
+ private static class TestServerSetWatcher implements ServerSetWatcher {
+
+ final LinkedBlockingQueue<ImmutableSet<DLSocketAddress>> changeQueue =
+ new LinkedBlockingQueue<ImmutableSet<DLSocketAddress>>();
+ final CopyOnWriteArrayList<ServerSetMonitor> monitors =
+ new CopyOnWriteArrayList<ServerSetMonitor>();
+
+ @Override
+ public void watch(ServerSetMonitor monitor) throws MonitorException {
+ monitors.add(monitor);
+ ImmutableSet<DLSocketAddress> change;
+ while ((change = changeQueue.poll()) != null) {
+ notifyChanges(change);
+ }
+ }
+
+ void notifyChanges(ImmutableSet<DLSocketAddress> addresses) {
+ if (monitors.isEmpty()) {
+ changeQueue.add(addresses);
+ } else {
+ for (ServerSetMonitor monitor : monitors) {
+ monitor.onChange(addresses);
+ }
+ }
+ }
+ }
+
+ @Test(timeout = 60000)
+ public void testPerformServerSetChangeOnServerSet() throws Exception {
+ TestServerSetWatcher serverSetWatcher = new TestServerSetWatcher();
+ ConsistentHashRoutingService routingService = new ConsistentHashRoutingService(
+ serverSetWatcher, 997, Integer.MAX_VALUE, NullStatsReceiver.get());
+
+ int basePort = 3180;
+ int numHosts = 4;
+ Set<DLSocketAddress> addresses1 = Sets.newConcurrentHashSet();
+ Set<DLSocketAddress> addresses2 = Sets.newConcurrentHashSet();
+ Set<DLSocketAddress> addresses3 = Sets.newConcurrentHashSet();
+
+ // fill up the addresses1
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+ DLSocketAddress dsa = new DLSocketAddress(i, inetAddress);
+ addresses1.add(dsa);
+ }
+ // fill up the addresses2 - overlap with addresses1
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + i);
+ DLSocketAddress dsa = new DLSocketAddress(i + 2, inetAddress);
+ addresses2.add(dsa);
+ }
+ // fill up the addresses3 - not overlap with addresses2
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
+ DLSocketAddress dsa = new DLSocketAddress(i, inetAddress);
+ addresses3.add(dsa);
+ }
+
+ final List<SocketAddress> leftAddresses = Lists.newArrayList();
+ final List<SocketAddress> joinAddresses = Lists.newArrayList();
+
+ RoutingService.RoutingListener routingListener = new RoutingService.RoutingListener() {
+ @Override
+ public void onServerLeft(SocketAddress address) {
+ synchronized (leftAddresses) {
+ leftAddresses.add(address);
+ leftAddresses.notifyAll();
+ }
+ }
+
+ @Override
+ public void onServerJoin(SocketAddress address) {
+ synchronized (joinAddresses) {
+ joinAddresses.add(address);
+ joinAddresses.notifyAll();
+ }
+ }
+ };
+
+ routingService.registerListener(routingListener);
+ serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses1));
+
+ routingService.startService();
+
+ synchronized (joinAddresses) {
+ while (joinAddresses.size() < numHosts) {
+ joinAddresses.wait();
+ }
+ }
+
+ // validate 4 nodes joined
+ synchronized (joinAddresses) {
+ assertEquals(numHosts, joinAddresses.size());
+ }
+ synchronized (leftAddresses) {
+ assertEquals(0, leftAddresses.size());
+ }
+ assertEquals(numHosts, routingService.shardId2Address.size());
+ assertEquals(numHosts, routingService.address2ShardId.size());
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+ assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+ int shardId = routingService.address2ShardId.get(inetAddress);
+ assertEquals(i, shardId);
+ SocketAddress sa = routingService.shardId2Address.get(shardId);
+ assertNotNull(sa);
+ assertEquals(inetAddress, sa);
+ }
+
+ // update addresses2 - 2 new hosts joined, 2 old hosts left
+ serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses2));
+ synchronized (joinAddresses) {
+ while (joinAddresses.size() < numHosts + 2) {
+ joinAddresses.wait();
+ }
+ }
+ synchronized (leftAddresses) {
+ while (leftAddresses.size() < 2) {
+ leftAddresses.wait();
+ }
+ }
+
+ assertEquals(numHosts + 2, routingService.shardId2Address.size());
+ assertEquals(numHosts + 2, routingService.address2ShardId.size());
+ // first 2 shards should not leave
+ for (int i = 0; i < 2; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+ assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+ int shardId = routingService.address2ShardId.get(inetAddress);
+ assertEquals(i, shardId);
+ SocketAddress sa = routingService.shardId2Address.get(shardId);
+ assertNotNull(sa);
+ assertEquals(inetAddress, sa);
+ }
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + i);
+ assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+ int shardId = routingService.address2ShardId.get(inetAddress);
+ assertEquals(i + 2, shardId);
+ SocketAddress sa = routingService.shardId2Address.get(shardId);
+ assertNotNull(sa);
+ assertEquals(inetAddress, sa);
+ }
+
+ // update addresses3
+ serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses3));
+ synchronized (joinAddresses) {
+ while (joinAddresses.size() < numHosts + 2 + numHosts) {
+ joinAddresses.wait();
+ }
+ }
+ synchronized (leftAddresses) {
+ while (leftAddresses.size() < 2 + numHosts) {
+ leftAddresses.wait();
+ }
+ }
+ assertEquals(numHosts + 2, routingService.shardId2Address.size());
+ assertEquals(numHosts + 2, routingService.address2ShardId.size());
+
+ // first 4 shards should leave
+ for (int i = 0; i < numHosts; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
+ assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+ int shardId = routingService.address2ShardId.get(inetAddress);
+ assertEquals(i, shardId);
+ SocketAddress sa = routingService.shardId2Address.get(shardId);
+ assertNotNull(sa);
+ assertEquals(inetAddress, sa);
+ }
+ // the other 2 shards should be still there
+ for (int i = 0; i < 2; i++) {
+ InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + 2 + i);
+ assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+ int shardId = routingService.address2ShardId.get(inetAddress);
+ assertEquals(numHosts + i, shardId);
+ SocketAddress sa = routingService.shardId2Address.get(shardId);
+ assertNotNull(sa);
+ assertEquals(inetAddress, sa);
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java
new file mode 100644
index 0000000..59665b9
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java
@@ -0,0 +1,73 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import com.google.common.collect.ImmutableSet;
+import com.twitter.common.net.pool.DynamicHostSet;
+import com.twitter.thrift.Endpoint;
+import com.twitter.thrift.ServiceInstance;
+import java.net.InetSocketAddress;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for `inet` name resolution.
+ */
+public class TestInetNameResolution {
+
+ private static final Logger logger = LoggerFactory.getLogger(TestRoutingService.class);
+
+ @Test(timeout = 10000)
+ public void testInetNameResolution() throws Exception {
+ String nameStr = "inet!127.0.0.1:3181";
+ final CountDownLatch resolved = new CountDownLatch(1);
+ final AtomicBoolean validationFailed = new AtomicBoolean(false);
+
+ NameServerSet serverSet = new NameServerSet(nameStr);
+ serverSet.watch(new DynamicHostSet.HostChangeMonitor<ServiceInstance>() {
+ @Override
+ public void onChange(ImmutableSet<ServiceInstance> hostSet) {
+ if (hostSet.size() > 1) {
+ logger.error("HostSet has more elements than expected {}", hostSet);
+ validationFailed.set(true);
+ resolved.countDown();
+ } else if (hostSet.size() == 1) {
+ ServiceInstance serviceInstance = hostSet.iterator().next();
+ Endpoint endpoint = serviceInstance.getAdditionalEndpoints().get("thrift");
+ InetSocketAddress address = new InetSocketAddress(endpoint.getHost(), endpoint.getPort());
+ if (endpoint.getPort() != 3181) {
+ logger.error("Port does not match the expected port {}", endpoint.getPort());
+ validationFailed.set(true);
+ } else if (!address.getAddress().getHostAddress().equals("127.0.0.1")) {
+ logger.error("Host address does not match the expected address {}",
+ address.getAddress().getHostAddress());
+ validationFailed.set(true);
+ }
+ resolved.countDown();
+ }
+ }
+ });
+
+ resolved.await();
+ Assert.assertEquals(false, validationFailed.get());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java
new file mode 100644
index 0000000..151663e
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java
@@ -0,0 +1,133 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import com.twitter.finagle.NoBrokersAvailableException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link RegionsRoutingService}.
+ */
+public class TestRegionsRoutingService {
+
+ @Test(timeout = 60000)
+ public void testRoutingListener() throws Exception {
+ int numRoutingServices = 5;
+ RoutingService.Builder[] routingServiceBuilders = new RoutingService.Builder[numRoutingServices];
+ Set<SocketAddress> hosts = new HashSet<SocketAddress>();
+ Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
+ for (int i = 0; i < numRoutingServices; i++) {
+ String finagleNameStr = "inet!127.0.0.1:" + (3181 + i);
+ routingServiceBuilders[i] = RoutingUtils.buildRoutingService(finagleNameStr);
+ SocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
+ hosts.add(address);
+ regionMap.put(address, "region-" + i);
+ }
+
+ final CountDownLatch doneLatch = new CountDownLatch(numRoutingServices);
+ final AtomicInteger numHostsLeft = new AtomicInteger(0);
+ final Set<SocketAddress> jointHosts = new HashSet<SocketAddress>();
+ RegionsRoutingService regionsRoutingService =
+ RegionsRoutingService.newBuilder()
+ .routingServiceBuilders(routingServiceBuilders)
+ .resolver(new DefaultRegionResolver(regionMap))
+ .build();
+ regionsRoutingService.registerListener(new RoutingService.RoutingListener() {
+ @Override
+ public void onServerLeft(SocketAddress address) {
+ numHostsLeft.incrementAndGet();
+ }
+
+ @Override
+ public void onServerJoin(SocketAddress address) {
+ jointHosts.add(address);
+ doneLatch.countDown();
+ }
+ });
+
+ regionsRoutingService.startService();
+
+ doneLatch.await();
+
+ assertEquals(numRoutingServices, jointHosts.size());
+ assertEquals(0, numHostsLeft.get());
+ assertTrue(Sets.difference(hosts, jointHosts).immutableCopy().isEmpty());
+ }
+
+ @Test(timeout = 60000)
+ public void testGetHost() throws Exception {
+ int numRoutingServices = 3;
+ RoutingService.Builder[] routingServiceBuilders = new RoutingService.Builder[numRoutingServices];
+ Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
+ for (int i = 0; i < numRoutingServices; i++) {
+ String finagleNameStr = "inet!127.0.0.1:" + (3181 + i);
+ routingServiceBuilders[i] = RoutingUtils.buildRoutingService(finagleNameStr);
+ SocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
+ regionMap.put(address, "region-" + i);
+ }
+
+ RegionsRoutingService regionsRoutingService =
+ RegionsRoutingService.newBuilder()
+ .resolver(new DefaultRegionResolver(regionMap))
+ .routingServiceBuilders(routingServiceBuilders)
+ .build();
+ regionsRoutingService.startService();
+
+ RoutingService.RoutingContext routingContext =
+ RoutingService.RoutingContext.of(new DefaultRegionResolver())
+ .addTriedHost(new InetSocketAddress("127.0.0.1", 3183), StatusCode.WRITE_EXCEPTION);
+ assertEquals(new InetSocketAddress("127.0.0.1", 3181),
+ regionsRoutingService.getHost("any", routingContext));
+
+ routingContext =
+ RoutingService.RoutingContext.of(new DefaultRegionResolver())
+ .addTriedHost(new InetSocketAddress("127.0.0.1", 3181), StatusCode.WRITE_EXCEPTION);
+ assertEquals(new InetSocketAddress("127.0.0.1", 3182),
+ regionsRoutingService.getHost("any", routingContext));
+
+ // add 3182 to routing context as tried host
+ routingContext.addTriedHost(new InetSocketAddress("127.0.0.1", 3182), StatusCode.WRITE_EXCEPTION);
+ assertEquals(new InetSocketAddress("127.0.0.1", 3183),
+ regionsRoutingService.getHost("any", routingContext));
+
+ // add 3183 to routing context as tried host
+ routingContext.addTriedHost(new InetSocketAddress("127.0.0.1", 3183), StatusCode.WRITE_EXCEPTION);
+ try {
+ regionsRoutingService.getHost("any", routingContext);
+ fail("Should fail to get host since all regions are tried.");
+ } catch (NoBrokersAvailableException nbae) {
+ // expected
+ }
+ }
+
+}
[24/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 5921233..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java
+++ /dev/null
@@ -1,631 +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.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;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 50b1405..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
+++ /dev/null
@@ -1,124 +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.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.metadata.LogMetadataStore;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
-
-import java.net.URI;
-import java.util.Iterator;
-import java.util.List;
-
-import static com.twitter.distributedlog.util.DLUtils.*;
-
-/**
- * ZooKeeper based log metadata store
- */
-public class ZKLogMetadataStore implements LogMetadataStore {
-
- final URI namespace;
- final Optional<URI> nsOptional;
- final ZooKeeperClient zkc;
- final ZKNamespaceWatcher nsWatcher;
-
- public ZKLogMetadataStore(
- DistributedLogConfiguration conf,
- URI namespace,
- ZooKeeperClient zkc,
- OrderedScheduler scheduler) {
- this.namespace = namespace;
- this.nsOptional = Optional.of(this.namespace);
- this.zkc = zkc;
- this.nsWatcher = new ZKNamespaceWatcher(conf, namespace, zkc, scheduler);
- }
-
- @Override
- public Future<URI> createLog(String logName) {
- return Future.value(namespace);
- }
-
- @Override
- public Future<Optional<URI>> getLogLocation(String logName) {
- return Future.value(nsOptional);
- }
-
- @Override
- public Future<Iterator<String>> getLogs() {
- final Promise<Iterator<String>> promise = new Promise<Iterator<String>>();
- final String nsRootPath = namespace.getPath();
- try {
- final ZooKeeper zk = zkc.get();
- zk.sync(nsRootPath, new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int syncRc, String syncPath, Object ctx) {
- if (KeeperException.Code.OK.intValue() == syncRc) {
- zk.getChildren(nsRootPath, false, new AsyncCallback.Children2Callback() {
- @Override
- public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
- if (KeeperException.Code.OK.intValue() == rc) {
- List<String> results = Lists.newArrayListWithExpectedSize(children.size());
- for (String child : children) {
- if (!isReservedStreamName(child)) {
- results.add(child);
- }
- }
- promise.setValue(results.iterator());
- } else if (KeeperException.Code.NONODE.intValue() == rc) {
- List<String> streams = Lists.newLinkedList();
- promise.setValue(streams.iterator());
- } else {
- promise.setException(new ZKException("Error reading namespace " + nsRootPath,
- KeeperException.Code.get(rc)));
- }
- }
- }, null);
- } else if (KeeperException.Code.NONODE.intValue() == syncRc) {
- List<String> streams = Lists.newLinkedList();
- promise.setValue(streams.iterator());
- } else {
- promise.setException(new ZKException("Error reading namespace " + nsRootPath,
- KeeperException.Code.get(syncRc)));
- }
- }
- }, null);
- zkc.get();
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- } catch (InterruptedException e) {
- promise.setException(e);
- }
- return promise;
- }
-
- @Override
- public void registerNamespaceListener(NamespaceListener listener) {
- this.nsWatcher.registerListener(listener);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentFilters.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentFilters.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentFilters.java
deleted file mode 100644
index e55b2f2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentFilters.java
+++ /dev/null
@@ -1,89 +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.DistributedLogConstants;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * Filters based on current zookeeper log segments.
- */
-public class ZKLogSegmentFilters {
-
- static final Logger LOG = LoggerFactory.getLogger(ZKLogSegmentFilters.class);
-
- /**
- * Write handler filter should return all inprogress log segments and the last completed log segment.
- * Because sequence id & ledger sequence number assignment rely on previous log segments.
- */
- public static final LogSegmentFilter WRITE_HANDLE_FILTER = new LogSegmentFilter() {
- @Override
- public Collection<String> filter(Collection<String> fullList) {
- List<String> result = new ArrayList<String>(fullList.size());
- String lastCompletedLogSegmentName = null;
- long lastLogSegmentSequenceNumber = -1L;
- for (String s : fullList) {
- if (s.startsWith(DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX)) {
- result.add(s);
- } else if (s.startsWith(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX)) {
- String[] parts = s.split("_");
- try {
- if (2 == parts.length) {
- // name: logrecs_<logsegment_sequence_number>
- long logSegmentSequenceNumber = Long.parseLong(parts[1]);
- if (logSegmentSequenceNumber > lastLogSegmentSequenceNumber) {
- lastLogSegmentSequenceNumber = logSegmentSequenceNumber;
- lastCompletedLogSegmentName = s;
- }
- } else if (6 == parts.length) {
- // name: logrecs_<start_tx_id>_<end_tx_id>_<logsegment_sequence_number>_<ledger_id>_<region_id>
- long logSegmentSequenceNumber = Long.parseLong(parts[3]);
- if (logSegmentSequenceNumber > lastLogSegmentSequenceNumber) {
- lastLogSegmentSequenceNumber = logSegmentSequenceNumber;
- lastCompletedLogSegmentName = s;
- }
- } else {
- // name: logrecs_<start_tx_id>_<end_tx_id> or any unknown names
- // we don't know the ledger sequence from the name, so add it to the list
- result.add(s);
- }
- } catch (NumberFormatException nfe) {
- LOG.warn("Unexpected sequence number in log segment {} :", s, nfe);
- result.add(s);
- }
- } else {
- LOG.error("Unknown log segment name : {}", s);
- }
- }
- if (null != lastCompletedLogSegmentName) {
- result.add(lastCompletedLogSegmentName);
- }
- if (LOG.isTraceEnabled()) {
- LOG.trace("Filtered log segments {} from {}.", result, fullList);
- }
- return result;
- }
- };
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentMetadataStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentMetadataStore.java
deleted file mode 100644
index 2076dd8..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentMetadataStore.java
+++ /dev/null
@@ -1,503 +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.google.common.collect.ImmutableList;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.callback.LogSegmentNamesListener;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.distributedlog.zk.DefaultZKOp;
-import com.twitter.distributedlog.zk.ZKOp;
-import com.twitter.distributedlog.zk.ZKTransaction;
-import com.twitter.distributedlog.zk.ZKVersionedSetOp;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.AsyncCallback.Children2Callback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * ZooKeeper based log segment metadata store.
- */
-public class ZKLogSegmentMetadataStore implements LogSegmentMetadataStore, Watcher, Children2Callback {
-
- private static final Logger logger = LoggerFactory.getLogger(ZKLogSegmentMetadataStore.class);
-
- private static final List<String> EMPTY_LIST = ImmutableList.of();
-
- private static class ReadLogSegmentsTask implements Runnable, FutureEventListener<Versioned<List<String>>> {
-
- private final String logSegmentsPath;
- private final ZKLogSegmentMetadataStore store;
- private int currentZKBackOffMs;
-
- ReadLogSegmentsTask(String logSegmentsPath,
- ZKLogSegmentMetadataStore metadataStore) {
- this.logSegmentsPath = logSegmentsPath;
- this.store = metadataStore;
- this.currentZKBackOffMs = store.minZKBackoffMs;
- }
-
- @Override
- public void onSuccess(final Versioned<List<String>> segments) {
- // reset the back off after a successful operation
- currentZKBackOffMs = store.minZKBackoffMs;
- store.notifyLogSegmentsUpdated(
- logSegmentsPath,
- store.listeners.get(logSegmentsPath),
- segments);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- int backoffMs;
- if (cause instanceof LogNotFoundException) {
- // the log segment has been deleted, remove all the registered listeners
- store.notifyLogStreamDeleted(logSegmentsPath,
- store.listeners.remove(logSegmentsPath));
- return;
- } else {
- backoffMs = currentZKBackOffMs;
- currentZKBackOffMs = Math.min(2 * currentZKBackOffMs, store.maxZKBackoffMs);
- }
- store.scheduleTask(logSegmentsPath, this, backoffMs);
- }
-
- @Override
- public void run() {
- if (null != store.listeners.get(logSegmentsPath)) {
- store.zkGetLogSegmentNames(logSegmentsPath, store).addEventListener(this);
- } else {
- logger.debug("Log segments listener for {} has been removed.", logSegmentsPath);
- }
- }
- }
-
- /**
- * A log segment names listener that keeps tracking the version of list of log segments that it has been notified.
- * It only notify the newer log segments.
- */
- static class VersionedLogSegmentNamesListener {
-
- private final LogSegmentNamesListener listener;
- private Versioned<List<String>> lastNotifiedLogSegments;
-
- VersionedLogSegmentNamesListener(LogSegmentNamesListener listener) {
- this.listener = listener;
- this.lastNotifiedLogSegments = new Versioned<List<String>>(EMPTY_LIST, Version.NEW);
- }
-
- synchronized void onSegmentsUpdated(Versioned<List<String>> logSegments) {
- if (lastNotifiedLogSegments.getVersion() == Version.NEW ||
- lastNotifiedLogSegments.getVersion().compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
- lastNotifiedLogSegments = logSegments;
- listener.onSegmentsUpdated(logSegments);
- }
- }
-
- @Override
- public int hashCode() {
- return listener.hashCode();
- }
-
- @Override
- public boolean equals(Object obj) {
- if (!(obj instanceof VersionedLogSegmentNamesListener)) {
- return false;
- }
- VersionedLogSegmentNamesListener other = (VersionedLogSegmentNamesListener) obj;
- return listener.equals(other.listener);
- }
-
- @Override
- public String toString() {
- return listener.toString();
- }
- }
-
- final DistributedLogConfiguration conf;
- // settings
- final int minZKBackoffMs;
- final int maxZKBackoffMs;
- final boolean skipMinVersionCheck;
-
- final ZooKeeperClient zkc;
- // log segment listeners
- final ConcurrentMap<String, Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener>> listeners;
- // scheduler
- final OrderedScheduler scheduler;
- final ReentrantReadWriteLock closeLock;
- boolean closed = false;
-
- public ZKLogSegmentMetadataStore(DistributedLogConfiguration conf,
- ZooKeeperClient zkc,
- OrderedScheduler scheduler) {
- this.conf = conf;
- this.zkc = zkc;
- this.listeners =
- new ConcurrentHashMap<String, Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener>>();
- this.scheduler = scheduler;
- this.closeLock = new ReentrantReadWriteLock();
- // settings
- this.minZKBackoffMs = conf.getZKRetryBackoffStartMillis();
- this.maxZKBackoffMs = conf.getZKRetryBackoffMaxMillis();
- this.skipMinVersionCheck = conf.getDLLedgerMetadataSkipMinVersionCheck();
- }
-
- protected void scheduleTask(Object key, Runnable r, long delayMs) {
- closeLock.readLock().lock();
- try {
- if (closed) {
- return;
- }
- scheduler.schedule(key, r, delayMs, TimeUnit.MILLISECONDS);
- } finally {
- closeLock.readLock().unlock();
- }
- }
-
- protected void submitTask(Object key, Runnable r) {
- closeLock.readLock().lock();
- try {
- if (closed) {
- return;
- }
- scheduler.submit(key, r);
- } finally {
- closeLock.readLock().unlock();
- }
- }
-
- // max sequence number and max transaction id
-
- @Override
- public void storeMaxLogSegmentSequenceNumber(Transaction<Object> txn,
- LogMetadata logMetadata,
- Versioned<Long> lssn,
- Transaction.OpListener<Version> listener) {
- Version version = lssn.getVersion();
- assert(version instanceof ZkVersion);
- ZkVersion zkVersion = (ZkVersion) version;
- byte[] data = DLUtils.serializeLogSegmentSequenceNumber(lssn.getValue());
- Op setDataOp = Op.setData(logMetadata.getLogSegmentsPath(), data, zkVersion.getZnodeVersion());
- ZKOp zkOp = new ZKVersionedSetOp(setDataOp, listener);
- txn.addOp(zkOp);
- }
-
- @Override
- public void storeMaxTxnId(Transaction<Object> txn,
- LogMetadataForWriter logMetadata,
- Versioned<Long> transactionId,
- Transaction.OpListener<Version> listener) {
- Version version = transactionId.getVersion();
- assert(version instanceof ZkVersion);
- ZkVersion zkVersion = (ZkVersion) version;
- byte[] data = DLUtils.serializeTransactionId(transactionId.getValue());
- Op setDataOp = Op.setData(logMetadata.getMaxTxIdPath(), data, zkVersion.getZnodeVersion());
- ZKOp zkOp = new ZKVersionedSetOp(setDataOp, listener);
- txn.addOp(zkOp);
- }
-
- // updates
-
- @Override
- public Transaction<Object> transaction() {
- return new ZKTransaction(zkc);
- }
-
- @Override
- public void createLogSegment(Transaction<Object> txn,
- LogSegmentMetadata segment,
- OpListener<Void> listener) {
- byte[] finalisedData = segment.getFinalisedData().getBytes(UTF_8);
- Op createOp = Op.create(
- segment.getZkPath(),
- finalisedData,
- zkc.getDefaultACL(),
- CreateMode.PERSISTENT);
- txn.addOp(DefaultZKOp.of(createOp, listener));
- }
-
- @Override
- public void deleteLogSegment(Transaction<Object> txn,
- final LogSegmentMetadata segment,
- final OpListener<Void> listener) {
- Op deleteOp = Op.delete(
- segment.getZkPath(),
- -1);
- logger.info("Delete segment : {}", segment);
- txn.addOp(DefaultZKOp.of(deleteOp, new OpListener<Void>() {
- @Override
- public void onCommit(Void r) {
- if (null != listener) {
- listener.onCommit(r);
- }
- }
-
- @Override
- public void onAbort(Throwable t) {
- logger.info("Aborted transaction on deleting segment {}", segment);
- KeeperException.Code kc;
- if (t instanceof KeeperException) {
- kc = ((KeeperException) t).code();
- } else if (t instanceof ZKException) {
- kc = ((ZKException) t).getKeeperExceptionCode();
- } else {
- abortListener(t);
- return;
- }
- if (KeeperException.Code.NONODE == kc) {
- abortListener(new LogSegmentNotFoundException(segment.getZkPath()));
- return;
- }
- abortListener(t);
- }
-
- private void abortListener(Throwable t) {
- if (null != listener) {
- listener.onAbort(t);
- }
- }
- }));
- }
-
- @Override
- public void updateLogSegment(Transaction<Object> txn, LogSegmentMetadata segment) {
- byte[] finalisedData = segment.getFinalisedData().getBytes(UTF_8);
- Op updateOp = Op.setData(segment.getZkPath(), finalisedData, -1);
- txn.addOp(DefaultZKOp.of(updateOp, null));
- }
-
- // reads
-
- /**
- * Process the watched events for registered listeners
- */
- @Override
- public void process(WatchedEvent event) {
- if (Event.EventType.None == event.getType()
- && Event.KeeperState.Expired == event.getState()) {
- Set<String> keySet = new HashSet<String>(listeners.keySet());
- for (String logSegmentsPath : keySet) {
- scheduleTask(logSegmentsPath, new ReadLogSegmentsTask(logSegmentsPath, this), 0L);
- }
- return;
- }
- String path = event.getPath();
- if (null == path) {
- return;
- }
- switch (event.getType()) {
- case NodeDeleted:
- notifyLogStreamDeleted(path, listeners.remove(path));
- break;
- case NodeChildrenChanged:
- new ReadLogSegmentsTask(path, this).run();
- break;
- default:
- break;
- }
- }
-
- @Override
- public Future<LogSegmentMetadata> getLogSegment(String logSegmentPath) {
- return LogSegmentMetadata.read(zkc, logSegmentPath, skipMinVersionCheck);
- }
-
- Future<Versioned<List<String>>> zkGetLogSegmentNames(String logSegmentsPath, Watcher watcher) {
- Promise<Versioned<List<String>>> result = new Promise<Versioned<List<String>>>();
- try {
- zkc.get().getChildren(logSegmentsPath, watcher, this, result);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- result.setException(FutureUtils.zkException(e, logSegmentsPath));
- } catch (InterruptedException e) {
- result.setException(FutureUtils.zkException(e, logSegmentsPath));
- }
- return result;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
- Promise<Versioned<List<String>>> result = ((Promise<Versioned<List<String>>>) ctx);
- if (KeeperException.Code.OK.intValue() == rc) {
- /** cversion: the number of changes to the children of this znode **/
- ZkVersion zkVersion = new ZkVersion(stat.getCversion());
- result.setValue(new Versioned(children, zkVersion));
- } else if (KeeperException.Code.NONODE.intValue() == rc) {
- result.setException(new LogNotFoundException("Log " + path + " not found"));
- } else {
- result.setException(new ZKException("Failed to get log segments from " + path,
- KeeperException.Code.get(rc)));
- }
- }
-
- @Override
- public Future<Versioned<List<String>>> getLogSegmentNames(String logSegmentsPath,
- LogSegmentNamesListener listener) {
- Watcher zkWatcher;
- if (null == listener) {
- zkWatcher = null;
- } else {
- closeLock.readLock().lock();
- try {
- if (closed) {
- zkWatcher = null;
- } else {
- Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listenerSet =
- listeners.get(logSegmentsPath);
- if (null == listenerSet) {
- Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> newListenerSet =
- new HashMap<LogSegmentNamesListener, VersionedLogSegmentNamesListener>();
- Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> oldListenerSet =
- listeners.putIfAbsent(logSegmentsPath, newListenerSet);
- if (null != oldListenerSet) {
- listenerSet = oldListenerSet;
- } else {
- listenerSet = newListenerSet;
- }
- }
- synchronized (listenerSet) {
- listenerSet.put(listener, new VersionedLogSegmentNamesListener(listener));
- if (!listeners.containsKey(logSegmentsPath)) {
- // listener set has been removed, add it back
- if (null != listeners.putIfAbsent(logSegmentsPath, listenerSet)) {
- logger.debug("Listener set is already found for log segments path {}", logSegmentsPath);
- }
- }
- }
- zkWatcher = ZKLogSegmentMetadataStore.this;
- }
- } finally {
- closeLock.readLock().unlock();
- }
- }
- Future<Versioned<List<String>>> getLogSegmentNamesResult = zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
- if (null != listener) {
- getLogSegmentNamesResult.addEventListener(new ReadLogSegmentsTask(logSegmentsPath, this));
- }
- return zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
- }
-
- @Override
- public void unregisterLogSegmentListener(String logSegmentsPath,
- LogSegmentNamesListener listener) {
- closeLock.readLock().lock();
- try {
- if (closed) {
- return;
- }
- Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listenerSet =
- listeners.get(logSegmentsPath);
- if (null == listenerSet) {
- return;
- }
- synchronized (listenerSet) {
- listenerSet.remove(listener);
- if (listenerSet.isEmpty()) {
- listeners.remove(logSegmentsPath, listenerSet);
- }
- }
- } finally {
- closeLock.readLock().unlock();
- }
- }
-
- @Override
- public void close() throws IOException {
- closeLock.writeLock().lock();
- try {
- if (closed) {
- return;
- }
- closed = true;
- } finally {
- closeLock.writeLock().unlock();
- }
- }
-
- // Notifications
-
- void notifyLogStreamDeleted(String logSegmentsPath,
- final Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listeners) {
- if (null == listeners) {
- return;
- }
- this.submitTask(logSegmentsPath, new Runnable() {
- @Override
- public void run() {
- for (LogSegmentNamesListener listener : listeners.keySet()) {
- listener.onLogStreamDeleted();
- }
- }
- });
-
- }
-
- void notifyLogSegmentsUpdated(String logSegmentsPath,
- final Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listeners,
- final Versioned<List<String>> segments) {
- if (null == listeners) {
- return;
- }
- this.submitTask(logSegmentsPath, new Runnable() {
- @Override
- public void run() {
- for (VersionedLogSegmentNamesListener listener : listeners.values()) {
- listener.onSegmentsUpdated(segments);
- }
- }
- });
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
deleted file mode 100644
index eeda804..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
+++ /dev/null
@@ -1,264 +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 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.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.data.Stat;
-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;
- protected Promise<Void> closePromise;
- protected final URI uri;
- // 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.
- protected final ZooKeeperClientBuilder writerZKCBuilder;
- protected final ZooKeeperClient writerZKC;
- protected final boolean ownWriterZKC;
- protected final ZooKeeperClientBuilder readerZKCBuilder;
- protected final ZooKeeperClient readerZKC;
- protected final boolean ownReaderZKC;
-
- ZKMetadataAccessor(String name,
- DistributedLogConfiguration conf,
- URI uri,
- ZooKeeperClientBuilder writerZKCBuilder,
- ZooKeeperClientBuilder readerZKCBuilder,
- StatsLogger statsLogger) {
- this.name = name;
- this.uri = uri;
-
- if (null == writerZKCBuilder) {
- RetryPolicy retryPolicy = null;
- if (conf.getZKNumRetries() > 0) {
- retryPolicy = new BoundExponentialBackoffRetryPolicy(
- conf.getZKRetryBackoffStartMillis(),
- conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
- }
- this.writerZKCBuilder = ZooKeeperClientBuilder.newBuilder()
- .name(String.format("dlzk:%s:dlm_writer_shared", name))
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .retryThreadCount(conf.getZKClientNumberRetryThreads())
- .requestRateLimit(conf.getZKRequestRateLimit())
- .zkAclId(conf.getZkAclId())
- .uri(uri)
- .retryPolicy(retryPolicy)
- .statsLogger(statsLogger.scope("dlzk_dlm_writer_shared"));
- this.ownWriterZKC = true;
- } else {
- this.writerZKCBuilder = writerZKCBuilder;
- this.ownWriterZKC = false;
- }
- this.writerZKC = this.writerZKCBuilder.build();
-
- if (null == readerZKCBuilder) {
- String zkServersForWriter = getZKServersFromDLUri(uri);
- String zkServersForReader;
- try {
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(this.writerZKC, uri);
- zkServersForReader = bkdlConfig.getDlZkServersForReader();
- } catch (IOException e) {
- LOG.warn("Error on resolving dl metadata bindings for {} : ", uri, e);
- zkServersForReader = zkServersForWriter;
- }
- if (zkServersForReader.equals(zkServersForWriter)) {
- LOG.info("Used same zookeeper servers '{}' for both writers and readers for {}.",
- zkServersForWriter, name);
- this.readerZKCBuilder = this.writerZKCBuilder;
- this.ownReaderZKC = false;
- } else {
- RetryPolicy retryPolicy = null;
- if (conf.getZKNumRetries() > 0) {
- retryPolicy = new BoundExponentialBackoffRetryPolicy(
- conf.getZKRetryBackoffStartMillis(),
- conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
- }
- this.readerZKCBuilder = ZooKeeperClientBuilder.newBuilder()
- .name(String.format("dlzk:%s:dlm_reader_shared", name))
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .retryThreadCount(conf.getZKClientNumberRetryThreads())
- .requestRateLimit(conf.getZKRequestRateLimit())
- .zkServers(zkServersForReader)
- .retryPolicy(retryPolicy)
- .zkAclId(conf.getZkAclId())
- .statsLogger(statsLogger.scope("dlzk_dlm_reader_shared"));
- this.ownReaderZKC = true;
- }
- } else {
- this.readerZKCBuilder = readerZKCBuilder;
- this.ownReaderZKC = false;
- }
- this.readerZKC = this.readerZKCBuilder.build();
- }
-
- /**
- * Get the name of the stream managed by this log manager
- *
- * @return streamName
- */
- @Override
- public String getStreamName() {
- return name;
- }
-
- /**
- * Creates or update the metadata stored at the node associated with the
- * name and URI
- * @param metadata opaque metadata to be stored for the node
- * @throws IOException
- */
- @Override
- public void createOrUpdateMetadata(byte[] metadata) throws IOException {
- checkClosedOrInError("createOrUpdateMetadata");
-
- String zkPath = getZKPath();
- LOG.debug("Setting application specific metadata on {}", zkPath);
- try {
- Stat currentStat = writerZKC.get().exists(zkPath, false);
- if (currentStat == null) {
- if (metadata.length > 0) {
- Utils.zkCreateFullPathOptimistic(writerZKC,
- zkPath,
- metadata,
- writerZKC.getDefaultACL(),
- CreateMode.PERSISTENT);
- }
- } else {
- writerZKC.get().setData(zkPath, metadata, currentStat.getVersion());
- }
- } catch (InterruptedException ie) {
- throw new DLInterruptedException("Interrupted on creating or updating container metadata", ie);
- } catch (Exception exc) {
- throw new IOException("Exception creating or updating container metadata", exc);
- }
- }
-
- /**
- * Delete the metadata stored at the associated node. This only deletes the metadata
- * and not the node itself
- * @throws IOException
- */
- @Override
- public void deleteMetadata() throws IOException {
- checkClosedOrInError("createOrUpdateMetadata");
- createOrUpdateMetadata(null);
- }
-
- /**
- * Retrieve the metadata stored at the node
- * @return byte array containing the metadata
- * @throws IOException
- */
- @Override
- public byte[] getMetadata() throws IOException {
- checkClosedOrInError("createOrUpdateMetadata");
- String zkPath = getZKPath();
- LOG.debug("Getting application specific metadata from {}", zkPath);
- try {
- Stat currentStat = readerZKC.get().exists(zkPath, false);
- if (currentStat == null) {
- return null;
- } else {
- return readerZKC.get().getData(zkPath, false, currentStat);
- }
- } catch (InterruptedException ie) {
- throw new DLInterruptedException("Error reading the max tx id from zk", ie);
- } catch (Exception e) {
- throw new IOException("Error reading the max tx id from zk", e);
- }
- }
-
- /**
- * Close the metadata accessor, freeing any resources it may hold.
- * @return future represents the close result.
- */
- @Override
- public Future<Void> asyncClose() {
- Promise<Void> closeFuture;
- synchronized (this) {
- if (null != closePromise) {
- return closePromise;
- }
- closeFuture = closePromise = new Promise<Void>();
- }
- // NOTE: ownWriterZKC and ownReaderZKC are mostly used by tests
- // the managers created by the namespace - whose zkc will be closed by namespace
- try {
- if (ownWriterZKC) {
- writerZKC.close();
- }
- if (ownReaderZKC) {
- readerZKC.close();
- }
- } catch (Exception e) {
- LOG.warn("Exception while closing distributed log manager", e);
- }
- FutureUtils.setValue(closeFuture, null);
- return closeFuture;
- }
-
- @Override
- public void close() throws IOException {
- FutureUtils.result(asyncClose());
- }
-
- public synchronized void checkClosedOrInError(String operation) throws AlreadyClosedException {
- if (null != closePromise) {
- throw new AlreadyClosedException("Executing " + operation + " on already closed ZKMetadataAccessor");
- }
- }
-
- protected String getZKPath() {
- return String.format("%s/%s", uri.getPath(), name);
- }
-
- @VisibleForTesting
- protected ZooKeeperClient getReaderZKC() {
- return readerZKC;
- }
-
- @VisibleForTesting
- protected ZooKeeperClient getWriterZKC() {
- return writerZKC;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 06bc8fb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
+++ /dev/null
@@ -1,133 +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.ZooKeeperClient;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.namespace.NamespaceWatcher;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static com.twitter.distributedlog.util.DLUtils.*;
-
-/**
- * Watcher on watching a given namespace
- */
-public class ZKNamespaceWatcher extends NamespaceWatcher
- implements Runnable, Watcher, AsyncCallback.Children2Callback {
-
- static final Logger logger = LoggerFactory.getLogger(ZKNamespaceWatcher.class);
-
- private final DistributedLogConfiguration conf;
- private final URI uri;
- private final ZooKeeperClient zkc;
- private final OrderedScheduler scheduler;
- private final AtomicBoolean namespaceWatcherSet = new AtomicBoolean(false);
-
- public ZKNamespaceWatcher(DistributedLogConfiguration conf,
- URI uri,
- ZooKeeperClient zkc,
- OrderedScheduler scheduler) {
- this.conf = conf;
- this.uri = uri;
- this.zkc = zkc;
- this.scheduler = scheduler;
- }
-
- private void scheduleTask(Runnable r, long ms) {
- try {
- scheduler.schedule(r, ms, TimeUnit.MILLISECONDS);
- } catch (RejectedExecutionException ree) {
- logger.error("Task {} scheduled in {} ms is rejected : ", new Object[]{r, ms, ree});
- }
- }
-
- @Override
- public void run() {
- try {
- doWatchNamespaceChanges();
- } catch (Exception e) {
- logger.error("Encountered unknown exception on watching namespace {} ", uri, e);
- }
- }
-
- public void watchNamespaceChanges() {
- if (!namespaceWatcherSet.compareAndSet(false, true)) {
- return;
- }
- doWatchNamespaceChanges();
- }
-
- private void doWatchNamespaceChanges() {
- try {
- zkc.get().getChildren(uri.getPath(), this, this, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
- } catch (InterruptedException e) {
- logger.warn("Interrupted on watching namespace changes for {} : ", uri, e);
- scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
- }
- }
-
- @Override
- public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
- if (KeeperException.Code.OK.intValue() == rc) {
- logger.info("Received updated logs under {} : {}", uri, children);
- List<String> result = new ArrayList<String>(children.size());
- for (String s : children) {
- if (isReservedStreamName(s)) {
- continue;
- }
- result.add(s);
- }
- for (NamespaceListener listener : listeners) {
- listener.onStreamsChanged(result.iterator());
- }
- } else {
- scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
- }
- }
-
- @Override
- public void process(WatchedEvent event) {
- if (event.getType() == Event.EventType.None) {
- if (event.getState() == Event.KeeperState.Expired) {
- scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
- }
- return;
- }
- if (event.getType() == Event.EventType.NodeChildrenChanged) {
- // watch namespace changes again.
- doWatchNamespaceChanges();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
deleted file mode 100644
index 8126723..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
+++ /dev/null
@@ -1,232 +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.acl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.thrift.AccessControlEntry;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TJSONProtocol;
-import org.apache.thrift.transport.TMemoryBuffer;
-import org.apache.thrift.transport.TMemoryInputTransport;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.data.Stat;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-public class ZKAccessControl {
-
- private static final int BUFFER_SIZE = 4096;
-
- public static final AccessControlEntry DEFAULT_ACCESS_CONTROL_ENTRY = new AccessControlEntry();
-
- public static class CorruptedAccessControlException extends IOException {
-
- private static final long serialVersionUID = 5391285182476211603L;
-
- public CorruptedAccessControlException(String zkPath, Throwable t) {
- super("Access Control @ " + zkPath + " is corrupted.", t);
- }
- }
-
- protected final AccessControlEntry accessControlEntry;
- protected final String zkPath;
- private int zkVersion;
-
- public ZKAccessControl(AccessControlEntry ace, String zkPath) {
- this(ace, zkPath, -1);
- }
-
- private ZKAccessControl(AccessControlEntry ace, String zkPath, int zkVersion) {
- this.accessControlEntry = ace;
- this.zkPath = zkPath;
- this.zkVersion = zkVersion;
- }
-
- @Override
- public int hashCode() {
- return Objects.hashCode(zkPath, accessControlEntry);
- }
-
- @Override
- public boolean equals(Object obj) {
- if (!(obj instanceof ZKAccessControl)) {
- return false;
- }
- ZKAccessControl other = (ZKAccessControl) obj;
- return Objects.equal(zkPath, other.zkPath) &&
- Objects.equal(accessControlEntry, other.accessControlEntry);
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("entry(path=").append(zkPath).append(", acl=")
- .append(accessControlEntry).append(")");
- return sb.toString();
- }
-
- @VisibleForTesting
- public String getZKPath() {
- return zkPath;
- }
-
- @VisibleForTesting
- public AccessControlEntry getAccessControlEntry() {
- return accessControlEntry;
- }
-
- public Future<ZKAccessControl> create(ZooKeeperClient zkc) {
- final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
- try {
- zkc.get().create(zkPath, serialize(accessControlEntry), zkc.getDefaultACL(), CreateMode.PERSISTENT,
- new AsyncCallback.StringCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, String name) {
- if (KeeperException.Code.OK.intValue() == rc) {
- ZKAccessControl.this.zkVersion = 0;
- promise.setValue(ZKAccessControl.this);
- } else {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- } catch (InterruptedException e) {
- promise.setException(e);
- } catch (IOException e) {
- promise.setException(e);
- }
- return promise;
- }
-
- public Future<ZKAccessControl> update(ZooKeeperClient zkc) {
- final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
- try {
- zkc.get().setData(zkPath, serialize(accessControlEntry), zkVersion, new AsyncCallback.StatCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, Stat stat) {
- if (KeeperException.Code.OK.intValue() == rc) {
- ZKAccessControl.this.zkVersion = stat.getVersion();
- promise.setValue(ZKAccessControl.this);
- } else {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- } catch (InterruptedException e) {
- promise.setException(e);
- } catch (IOException e) {
- promise.setException(e);
- }
- return promise;
- }
-
- public static Future<ZKAccessControl> read(final ZooKeeperClient zkc, final String zkPath, Watcher watcher) {
- final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
-
- try {
- zkc.get().getData(zkPath, watcher, new AsyncCallback.DataCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
- if (KeeperException.Code.OK.intValue() == rc) {
- try {
- AccessControlEntry ace = deserialize(zkPath, data);
- promise.setValue(new ZKAccessControl(ace, zkPath, stat.getVersion()));
- } catch (IOException ioe) {
- promise.setException(ioe);
- }
- } else {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- } catch (InterruptedException e) {
- promise.setException(e);
- }
- return promise;
- }
-
- public static Future<Void> delete(final ZooKeeperClient zkc, final String zkPath) {
- final Promise<Void> promise = new Promise<Void>();
-
- try {
- zkc.get().delete(zkPath, -1, new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx) {
- if (KeeperException.Code.OK.intValue() == rc ||
- KeeperException.Code.NONODE.intValue() == rc) {
- promise.setValue(null);
- } else {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- } catch (InterruptedException e) {
- promise.setException(e);
- }
- return promise;
- }
-
- static byte[] serialize(AccessControlEntry ace) throws IOException {
- TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
- TJSONProtocol protocol = new TJSONProtocol(transport);
- try {
- ace.write(protocol);
- transport.flush();
- return transport.toString(UTF_8.name()).getBytes(UTF_8);
- } catch (TException e) {
- throw new IOException("Failed to serialize access control entry : ", e);
- } catch (UnsupportedEncodingException uee) {
- throw new IOException("Failed to serialize acesss control entry : ", uee);
- }
- }
-
- static AccessControlEntry deserialize(String zkPath, byte[] data) throws IOException {
- if (data.length == 0) {
- return DEFAULT_ACCESS_CONTROL_ENTRY;
- }
-
- AccessControlEntry ace = new AccessControlEntry();
- TMemoryInputTransport transport = new TMemoryInputTransport(data);
- TJSONProtocol protocol = new TJSONProtocol(transport);
- try {
- ace.read(protocol);
- } catch (TException e) {
- throw new CorruptedAccessControlException(zkPath, e);
- }
- return ace;
- }
-
-}
[43/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
new file mode 100644
index 0000000..b3f3368
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
@@ -0,0 +1,486 @@
+/**
+ * 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 org.apache.distributedlog.client;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.base.Ticker;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecordSet;
+import org.apache.distributedlog.LogRecordSetBuffer;
+import org.apache.distributedlog.client.speculative.DefaultSpeculativeRequestExecutionPolicy;
+import org.apache.distributedlog.client.speculative.SpeculativeRequestExecutionPolicy;
+import org.apache.distributedlog.client.speculative.SpeculativeRequestExecutor;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.service.DistributedLogClient;
+import com.twitter.finagle.IndividualRequestTimeoutException;
+import com.twitter.util.Duration;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Write to multiple streams.
+ */
+public class DistributedLogMultiStreamWriter implements Runnable {
+
+ /**
+ * Create a new builder to create a multi stream writer.
+ *
+ * @return a new builder to create a multi stream writer.
+ */
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ /**
+ * Builder for the multi stream writer.
+ */
+ public static class Builder {
+
+ private DistributedLogClient client = null;
+ private List<String> streams = null;
+ private int bufferSize = 16 * 1024; // 16k
+ private long flushIntervalMicros = 2000; // 2ms
+ private CompressionCodec.Type codec = CompressionCodec.Type.NONE;
+ private ScheduledExecutorService executorService = null;
+ private long requestTimeoutMs = 500; // 500ms
+ private int firstSpeculativeTimeoutMs = 50; // 50ms
+ private int maxSpeculativeTimeoutMs = 200; // 200ms
+ private float speculativeBackoffMultiplier = 2;
+ private Ticker ticker = Ticker.systemTicker();
+
+ private Builder() {}
+
+ /**
+ * Set the distributedlog client used for multi stream writer.
+ *
+ * @param client
+ * distributedlog client
+ * @return builder
+ */
+ public Builder client(DistributedLogClient client) {
+ this.client = client;
+ return this;
+ }
+
+ /**
+ * Set the list of streams to write to.
+ *
+ * @param streams
+ * list of streams to write
+ * @return builder
+ */
+ public Builder streams(List<String> streams) {
+ this.streams = streams;
+ return this;
+ }
+
+ /**
+ * Set the output buffer size.
+ *
+ * <p>If output buffer size is 0, the writes will be transmitted to
+ * wire immediately.
+ *
+ * @param bufferSize
+ * output buffer size
+ * @return builder
+ */
+ public Builder bufferSize(int bufferSize) {
+ this.bufferSize = bufferSize;
+ return this;
+ }
+
+ /**
+ * Set the flush interval in milliseconds.
+ *
+ * @param flushIntervalMs
+ * flush interval in milliseconds.
+ * @return builder
+ */
+ public Builder flushIntervalMs(int flushIntervalMs) {
+ this.flushIntervalMicros = TimeUnit.MILLISECONDS.toMicros(flushIntervalMs);
+ return this;
+ }
+
+ /**
+ * Set the flush interval in microseconds.
+ *
+ * @param flushIntervalMicros
+ * flush interval in microseconds.
+ * @return builder
+ */
+ public Builder flushIntervalMicros(int flushIntervalMicros) {
+ this.flushIntervalMicros = flushIntervalMicros;
+ return this;
+ }
+
+ /**
+ * Set compression codec.
+ *
+ * @param codec compression codec.
+ * @return builder
+ */
+ public Builder compressionCodec(CompressionCodec.Type codec) {
+ this.codec = codec;
+ return this;
+ }
+
+ /**
+ * Set the scheduler to flush output buffers.
+ *
+ * @param executorService
+ * executor service to flush output buffers.
+ * @return builder
+ */
+ public Builder scheduler(ScheduledExecutorService executorService) {
+ this.executorService = executorService;
+ return this;
+ }
+
+ /**
+ * Set request timeout in milliseconds.
+ *
+ * @param requestTimeoutMs
+ * request timeout in milliseconds.
+ * @return builder
+ */
+ public Builder requestTimeoutMs(long requestTimeoutMs) {
+ this.requestTimeoutMs = requestTimeoutMs;
+ return this;
+ }
+
+ /**
+ * Set the first speculative timeout in milliseconds.
+ *
+ * <p>The multi-streams writer does speculative writes on streams.
+ * The write issues first write request to a stream, if the write request
+ * doesn't respond within speculative timeout. it issues next write request
+ * to a different stream. It does such speculative retries until receive
+ * a success or request timeout ({@link #requestTimeoutMs(long)}).
+ *
+ * <p>This setting is to configure the first speculative timeout, in milliseconds.
+ *
+ * @param timeoutMs
+ * timeout in milliseconds
+ * @return builder
+ */
+ public Builder firstSpeculativeTimeoutMs(int timeoutMs) {
+ this.firstSpeculativeTimeoutMs = timeoutMs;
+ return this;
+ }
+
+ /**
+ * Set the max speculative timeout in milliseconds.
+ *
+ * <p>The multi-streams writer does speculative writes on streams.
+ * The write issues first write request to a stream, if the write request
+ * doesn't respond within speculative timeout. it issues next write request
+ * to a different stream. It does such speculative retries until receive
+ * a success or request timeout ({@link #requestTimeoutMs(long)}).
+ *
+ * <p>This setting is to configure the max speculative timeout, in milliseconds.
+ *
+ * @param timeoutMs
+ * timeout in milliseconds
+ * @return builder
+ */
+ public Builder maxSpeculativeTimeoutMs(int timeoutMs) {
+ this.maxSpeculativeTimeoutMs = timeoutMs;
+ return this;
+ }
+
+ /**
+ * Set the speculative timeout backoff multiplier.
+ *
+ * <p>The multi-streams writer does speculative writes on streams.
+ * The write issues first write request to a stream, if the write request
+ * doesn't respond within speculative timeout. it issues next write request
+ * to a different stream. It does such speculative retries until receive
+ * a success or request timeout ({@link #requestTimeoutMs(long)}).
+ *
+ * <p>This setting is to configure the speculative timeout backoff multiplier.
+ *
+ * @param multiplier
+ * backoff multiplier
+ * @return builder
+ */
+ public Builder speculativeBackoffMultiplier(float multiplier) {
+ this.speculativeBackoffMultiplier = multiplier;
+ return this;
+ }
+
+ /**
+ * Ticker for timing.
+ *
+ * @param ticker
+ * ticker
+ * @return builder
+ * @see Ticker
+ */
+ public Builder clockTicker(Ticker ticker) {
+ this.ticker = ticker;
+ return this;
+ }
+
+ /**
+ * Build the multi stream writer.
+ *
+ * @return the multi stream writer.
+ */
+ public DistributedLogMultiStreamWriter build() {
+ checkArgument((null != streams && !streams.isEmpty()),
+ "No streams provided");
+ checkNotNull(client,
+ "No distributedlog client provided");
+ checkNotNull(codec,
+ "No compression codec provided");
+ checkArgument(firstSpeculativeTimeoutMs > 0
+ && firstSpeculativeTimeoutMs <= maxSpeculativeTimeoutMs
+ && speculativeBackoffMultiplier > 0
+ && maxSpeculativeTimeoutMs < requestTimeoutMs,
+ "Invalid speculative timeout settings");
+ return new DistributedLogMultiStreamWriter(
+ streams,
+ client,
+ Math.min(bufferSize, MAX_LOGRECORDSET_SIZE),
+ flushIntervalMicros,
+ requestTimeoutMs,
+ firstSpeculativeTimeoutMs,
+ maxSpeculativeTimeoutMs,
+ speculativeBackoffMultiplier,
+ codec,
+ ticker,
+ executorService);
+ }
+ }
+
+ /**
+ * Pending Write Request.
+ */
+ class PendingWriteRequest implements FutureEventListener<DLSN>,
+ SpeculativeRequestExecutor {
+
+ private final LogRecordSetBuffer recordSet;
+ private AtomicBoolean complete = new AtomicBoolean(false);
+ private final Stopwatch stopwatch = Stopwatch.createStarted(clockTicker);
+ private int nextStream;
+ private int numTriedStreams = 0;
+
+ PendingWriteRequest(LogRecordSetBuffer recordSet) {
+ this.recordSet = recordSet;
+ this.nextStream = Math.abs(nextStreamId.incrementAndGet()) % numStreams;
+ }
+
+ synchronized String sendNextWrite() {
+ long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
+ if (elapsedMs > requestTimeoutMs || numTriedStreams >= numStreams) {
+ fail(new IndividualRequestTimeoutException(Duration.fromMilliseconds(elapsedMs)));
+ return null;
+ }
+ try {
+ return sendWriteToStream(nextStream);
+ } finally {
+ nextStream = (nextStream + 1) % numStreams;
+ ++numTriedStreams;
+ }
+ }
+
+ synchronized String sendWriteToStream(int streamId) {
+ String stream = getStream(streamId);
+ client.writeRecordSet(stream, recordSet)
+ .addEventListener(this);
+ return stream;
+ }
+
+ @Override
+ public void onSuccess(DLSN dlsn) {
+ if (!complete.compareAndSet(false, true)) {
+ return;
+ }
+ recordSet.completeTransmit(
+ dlsn.getLogSegmentSequenceNo(),
+ dlsn.getEntryId(),
+ dlsn.getSlotId());
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ sendNextWrite();
+ }
+
+ private void fail(Throwable cause) {
+ if (!complete.compareAndSet(false, true)) {
+ return;
+ }
+ recordSet.abortTransmit(cause);
+ }
+
+ @Override
+ public Future<Boolean> issueSpeculativeRequest() {
+ return Future.value(!complete.get() && null != sendNextWrite());
+ }
+ }
+
+ private final int numStreams;
+ private final List<String> streams;
+ private final DistributedLogClient client;
+ private final int bufferSize;
+ private final long requestTimeoutMs;
+ private final SpeculativeRequestExecutionPolicy speculativePolicy;
+ private final Ticker clockTicker;
+ private final CompressionCodec.Type codec;
+ private final ScheduledExecutorService scheduler;
+ private final boolean ownScheduler;
+ private final AtomicInteger nextStreamId;
+ private LogRecordSet.Writer recordSetWriter;
+
+ private DistributedLogMultiStreamWriter(List<String> streams,
+ DistributedLogClient client,
+ int bufferSize,
+ long flushIntervalMicros,
+ long requestTimeoutMs,
+ int firstSpecultiveTimeoutMs,
+ int maxSpeculativeTimeoutMs,
+ float speculativeBackoffMultiplier,
+ CompressionCodec.Type codec,
+ Ticker clockTicker,
+ ScheduledExecutorService scheduler) {
+ this.streams = Lists.newArrayList(streams);
+ this.numStreams = this.streams.size();
+ this.client = client;
+ this.bufferSize = bufferSize;
+ this.requestTimeoutMs = requestTimeoutMs;
+ this.codec = codec;
+ this.clockTicker = clockTicker;
+ if (null == scheduler) {
+ this.scheduler = Executors.newSingleThreadScheduledExecutor(
+ new ThreadFactoryBuilder()
+ .setDaemon(true)
+ .setNameFormat("MultiStreamWriterFlushThread-%d")
+ .build());
+ this.ownScheduler = true;
+ } else {
+ this.scheduler = scheduler;
+ this.ownScheduler = false;
+ }
+ this.speculativePolicy = new DefaultSpeculativeRequestExecutionPolicy(
+ firstSpecultiveTimeoutMs,
+ maxSpeculativeTimeoutMs,
+ speculativeBackoffMultiplier);
+ // shuffle the streams
+ Collections.shuffle(this.streams);
+ this.nextStreamId = new AtomicInteger(0);
+ this.recordSetWriter = newRecordSetWriter();
+
+ if (flushIntervalMicros > 0) {
+ this.scheduler.scheduleAtFixedRate(
+ this,
+ flushIntervalMicros,
+ flushIntervalMicros,
+ TimeUnit.MICROSECONDS);
+ }
+ }
+
+ String getStream(int streamId) {
+ return streams.get(streamId);
+ }
+
+ synchronized LogRecordSet.Writer getLogRecordSetWriter() {
+ return recordSetWriter;
+ }
+
+ private LogRecordSet.Writer newRecordSetWriter() {
+ return LogRecordSet.newWriter(
+ bufferSize,
+ codec);
+ }
+
+ public synchronized Future<DLSN> write(ByteBuffer buffer) {
+ int logRecordSize = buffer.remaining();
+ if (logRecordSize > MAX_LOGRECORD_SIZE) {
+ return Future.exception(new LogRecordTooLongException(
+ "Log record of size " + logRecordSize + " written when only "
+ + MAX_LOGRECORD_SIZE + " is allowed"));
+ }
+ // if exceed max number of bytes
+ if ((recordSetWriter.getNumBytes() + logRecordSize) > MAX_LOGRECORDSET_SIZE) {
+ flush();
+ }
+ Promise<DLSN> writePromise = new Promise<DLSN>();
+ try {
+ recordSetWriter.writeRecord(buffer, writePromise);
+ } catch (LogRecordTooLongException e) {
+ return Future.exception(e);
+ } catch (WriteException e) {
+ recordSetWriter.abortTransmit(e);
+ recordSetWriter = newRecordSetWriter();
+ return Future.exception(e);
+ }
+ if (recordSetWriter.getNumBytes() >= bufferSize) {
+ flush();
+ }
+ return writePromise;
+ }
+
+ @Override
+ public void run() {
+ flush();
+ }
+
+ private void flush() {
+ LogRecordSet.Writer recordSetToFlush;
+ synchronized (this) {
+ if (recordSetWriter.getNumRecords() == 0) {
+ return;
+ }
+ recordSetToFlush = recordSetWriter;
+ recordSetWriter = newRecordSetWriter();
+ }
+ transmit(recordSetToFlush);
+ }
+
+ private void transmit(LogRecordSet.Writer recordSetToFlush) {
+ PendingWriteRequest writeRequest =
+ new PendingWriteRequest(recordSetToFlush);
+ this.speculativePolicy.initiateSpeculativeRequest(scheduler, writeRequest);
+ }
+
+ public void close() {
+ if (ownScheduler) {
+ this.scheduler.shutdown();
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java
new file mode 100644
index 0000000..ed6269b
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java
@@ -0,0 +1,68 @@
+/**
+ * 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 org.apache.distributedlog.client.monitor;
+
+import com.twitter.util.Future;
+import java.net.SocketAddress;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Interface for distributedlog monitor service.
+ */
+public interface MonitorServiceClient {
+
+ /**
+ * Check a given stream.
+ *
+ * @param stream
+ * stream.
+ * @return check result.
+ */
+ Future<Void> check(String stream);
+
+ /**
+ * Send heartbeat to the stream and its readers.
+ *
+ * @param stream
+ * stream.
+ * @return check result.
+ */
+ Future<Void> heartbeat(String stream);
+
+ /**
+ * Get current ownership distribution from current monitor service view.
+ *
+ * @return current ownership distribution
+ */
+ Map<SocketAddress, Set<String>> getStreamOwnershipDistribution();
+
+ /**
+ * Enable/Disable accepting new stream on a given proxy.
+ *
+ * @param enabled
+ * flag to enable/disable accepting new streams on a given proxy
+ * @return void
+ */
+ Future<Void> setAcceptNewStream(boolean enabled);
+
+ /**
+ * Close the client.
+ */
+ void close();
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java
new file mode 100644
index 0000000..d7e2c94
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * DistributedLog Monitor Client.
+ */
+package org.apache.distributedlog.client.monitor;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java
new file mode 100644
index 0000000..f3c24ca
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java
@@ -0,0 +1,235 @@
+/**
+ * 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 org.apache.distributedlog.client.ownership;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.stats.OwnershipStatsLogger;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Client Side Ownership Cache.
+ */
+public class OwnershipCache implements TimerTask {
+
+ private static final Logger logger = LoggerFactory.getLogger(OwnershipCache.class);
+
+ private final ConcurrentHashMap<String, SocketAddress> stream2Addresses =
+ new ConcurrentHashMap<String, SocketAddress>();
+ private final ConcurrentHashMap<SocketAddress, Set<String>> address2Streams =
+ new ConcurrentHashMap<SocketAddress, Set<String>>();
+ private final ClientConfig clientConfig;
+ private final HashedWheelTimer timer;
+
+ // Stats
+ private final OwnershipStatsLogger ownershipStatsLogger;
+
+ public OwnershipCache(ClientConfig clientConfig,
+ HashedWheelTimer timer,
+ StatsReceiver statsReceiver,
+ StatsReceiver streamStatsReceiver) {
+ this.clientConfig = clientConfig;
+ this.timer = timer;
+ this.ownershipStatsLogger = new OwnershipStatsLogger(statsReceiver, streamStatsReceiver);
+ scheduleDumpOwnershipCache();
+ }
+
+ private void scheduleDumpOwnershipCache() {
+ if (clientConfig.isPeriodicDumpOwnershipCacheEnabled()
+ && clientConfig.getPeriodicDumpOwnershipCacheIntervalMs() > 0) {
+ timer.newTimeout(this, clientConfig.getPeriodicDumpOwnershipCacheIntervalMs(),
+ TimeUnit.MILLISECONDS);
+ }
+ }
+
+ @Override
+ public void run(Timeout timeout) throws Exception {
+ if (timeout.isCancelled()) {
+ return;
+ }
+ logger.info("Ownership cache : {} streams cached, {} hosts cached",
+ stream2Addresses.size(), address2Streams.size());
+ logger.info("Cached streams : {}", stream2Addresses);
+ scheduleDumpOwnershipCache();
+ }
+
+ public OwnershipStatsLogger getOwnershipStatsLogger() {
+ return ownershipStatsLogger;
+ }
+
+ /**
+ * Update ownership of <i>stream</i> to <i>addr</i>.
+ *
+ * @param stream
+ * Stream Name.
+ * @param addr
+ * Owner Address.
+ * @return true if owner is updated
+ */
+ public boolean updateOwner(String stream, SocketAddress addr) {
+ // update ownership
+ SocketAddress oldAddr = stream2Addresses.putIfAbsent(stream, addr);
+ if (null != oldAddr && oldAddr.equals(addr)) {
+ return true;
+ }
+ if (null != oldAddr) {
+ if (stream2Addresses.replace(stream, oldAddr, addr)) {
+ // Store the relevant mappings for this topic and host combination
+ logger.info("Storing ownership for stream : {}, old host : {}, new host : {}.",
+ new Object[] { stream, oldAddr, addr });
+ StringBuilder sb = new StringBuilder();
+ sb.append("Ownership changed '")
+ .append(oldAddr).append("' -> '").append(addr).append("'");
+ removeOwnerFromStream(stream, oldAddr, sb.toString());
+
+ // update stats
+ ownershipStatsLogger.onRemove(stream);
+ ownershipStatsLogger.onAdd(stream);
+ } else {
+ logger.warn("Ownership of stream : {} has been changed from {} to {} when storing host : {}.",
+ new Object[] { stream, oldAddr, stream2Addresses.get(stream), addr });
+ return false;
+ }
+ } else {
+ logger.info("Storing ownership for stream : {}, host : {}.", stream, addr);
+ // update stats
+ ownershipStatsLogger.onAdd(stream);
+ }
+
+ Set<String> streamsForHost = address2Streams.get(addr);
+ if (null == streamsForHost) {
+ Set<String> newStreamsForHost = new HashSet<String>();
+ streamsForHost = address2Streams.putIfAbsent(addr, newStreamsForHost);
+ if (null == streamsForHost) {
+ streamsForHost = newStreamsForHost;
+ }
+ }
+ synchronized (streamsForHost) {
+ // check whether the ownership changed, since it might happend after replace succeed
+ if (addr.equals(stream2Addresses.get(stream))) {
+ streamsForHost.add(stream);
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Get the cached owner for stream <code>stream</code>.
+ *
+ * @param stream
+ * stream to lookup ownership
+ * @return owner's address
+ */
+ public SocketAddress getOwner(String stream) {
+ SocketAddress address = stream2Addresses.get(stream);
+ if (null == address) {
+ ownershipStatsLogger.onMiss(stream);
+ } else {
+ ownershipStatsLogger.onHit(stream);
+ }
+ return address;
+ }
+
+ /**
+ * Remove the owner <code>addr</code> from <code>stream</code> for a given <code>reason</code>.
+ *
+ * @param stream stream name
+ * @param addr owner address
+ * @param reason reason to remove ownership
+ */
+ public void removeOwnerFromStream(String stream, SocketAddress addr, String reason) {
+ if (stream2Addresses.remove(stream, addr)) {
+ logger.info("Removed stream to host mapping for (stream: {} -> host: {}) : reason = '{}'.",
+ new Object[] { stream, addr, reason });
+ }
+ Set<String> streamsForHost = address2Streams.get(addr);
+ if (null != streamsForHost) {
+ synchronized (streamsForHost) {
+ if (streamsForHost.remove(stream)) {
+ logger.info("Removed stream ({}) from host {} : reason = '{}'.",
+ new Object[] { stream, addr, reason });
+ if (streamsForHost.isEmpty()) {
+ address2Streams.remove(addr, streamsForHost);
+ }
+ ownershipStatsLogger.onRemove(stream);
+ }
+ }
+ }
+ }
+
+ /**
+ * Remove all streams from host <code>addr</code>.
+ *
+ * @param addr
+ * host to remove ownerships
+ */
+ public void removeAllStreamsFromOwner(SocketAddress addr) {
+ logger.info("Remove streams mapping for host {}", addr);
+ Set<String> streamsForHost = address2Streams.get(addr);
+ if (null != streamsForHost) {
+ synchronized (streamsForHost) {
+ for (String s : streamsForHost) {
+ if (stream2Addresses.remove(s, addr)) {
+ logger.info("Removing mapping for stream : {} from host : {}", s, addr);
+ ownershipStatsLogger.onRemove(s);
+ }
+ }
+ address2Streams.remove(addr, streamsForHost);
+ }
+ }
+ }
+
+ /**
+ * Get the number cached streams.
+ *
+ * @return number cached streams.
+ */
+ public int getNumCachedStreams() {
+ return stream2Addresses.size();
+ }
+
+ /**
+ * Get the stream ownership distribution across proxies.
+ *
+ * @return stream ownership distribution
+ */
+ public Map<SocketAddress, Set<String>> getStreamOwnershipDistribution() {
+ return ImmutableMap.copyOf(address2Streams);
+ }
+
+ /**
+ * Get the stream ownership mapping.
+ *
+ * @return stream ownership mapping.
+ */
+ public Map<String, SocketAddress> getStreamOwnerMapping() {
+ return stream2Addresses;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java
new file mode 100644
index 0000000..486bd6f
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Utils for managing ownership at client side.
+ */
+package org.apache.distributedlog.client.ownership;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/package-info.java
new file mode 100644
index 0000000..d22b0da
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * DistributedLog Client.
+ */
+package org.apache.distributedlog.client;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java
new file mode 100644
index 0000000..9b5c7f6
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java
@@ -0,0 +1,51 @@
+/**
+ * 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 org.apache.distributedlog.client.proxy;
+
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import com.twitter.finagle.Service;
+import com.twitter.finagle.thrift.ThriftClientRequest;
+import com.twitter.util.Future;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Cluster client.
+ */
+public class ClusterClient {
+
+ private final Service<ThriftClientRequest, byte[]> client;
+ private final DistributedLogService.ServiceIface service;
+
+ public ClusterClient(Service<ThriftClientRequest, byte[]> client,
+ DistributedLogService.ServiceIface service) {
+ this.client = client;
+ this.service = service;
+ }
+
+ public Service<ThriftClientRequest, byte[]> getClient() {
+ return client;
+ }
+
+ public DistributedLogService.ServiceIface getService() {
+ return service;
+ }
+
+ public Future<BoxedUnit> close() {
+ return client.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java
new file mode 100644
index 0000000..769cca8
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java
@@ -0,0 +1,35 @@
+/**
+ * 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 org.apache.distributedlog.client.proxy;
+
+import java.net.SocketAddress;
+import java.util.Set;
+
+/**
+ * Provider to provider list of hosts for handshaking.
+ */
+public interface HostProvider {
+
+ /**
+ * Get the list of hosts for handshaking.
+ *
+ * @return list of hosts for handshaking.
+ */
+ Set<SocketAddress> getHosts();
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java
new file mode 100644
index 0000000..6ef1d8e
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java
@@ -0,0 +1,165 @@
+/**
+ * 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 org.apache.distributedlog.client.proxy;
+
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.stats.ClientStats;
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import com.twitter.finagle.Service;
+import com.twitter.finagle.ThriftMux;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId;
+import com.twitter.finagle.thrift.ThriftClientFramedCodec;
+import com.twitter.finagle.thrift.ThriftClientRequest;
+import com.twitter.util.Duration;
+import com.twitter.util.Future;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Client talks to a single proxy.
+ */
+public class ProxyClient {
+
+ /**
+ * Builder to build a proxy client talking to given host <code>address</code>.
+ */
+ public interface Builder {
+ /**
+ * Build a proxy client to <code>address</code>.
+ *
+ * @param address
+ * proxy address
+ * @return proxy client
+ */
+ ProxyClient build(SocketAddress address);
+ }
+
+ public static Builder newBuilder(String clientName,
+ ClientId clientId,
+ ClientBuilder clientBuilder,
+ ClientConfig clientConfig,
+ ClientStats clientStats) {
+ return new DefaultBuilder(clientName, clientId, clientBuilder, clientConfig, clientStats);
+ }
+
+ /**
+ * Default Builder for {@link ProxyClient}.
+ */
+ public static class DefaultBuilder implements Builder {
+
+ private final String clientName;
+ private final ClientId clientId;
+ private final ClientBuilder clientBuilder;
+ private final ClientStats clientStats;
+
+ private DefaultBuilder(String clientName,
+ ClientId clientId,
+ ClientBuilder clientBuilder,
+ ClientConfig clientConfig,
+ ClientStats clientStats) {
+ this.clientName = clientName;
+ this.clientId = clientId;
+ this.clientStats = clientStats;
+ // client builder
+ ClientBuilder builder = setDefaultSettings(
+ null == clientBuilder ? getDefaultClientBuilder(clientConfig) : clientBuilder);
+ this.clientBuilder = configureThriftMux(builder, clientId, clientConfig);
+ }
+
+ @SuppressWarnings("unchecked")
+ private ClientBuilder configureThriftMux(ClientBuilder builder,
+ ClientId clientId,
+ ClientConfig clientConfig) {
+ if (clientConfig.getThriftMux()) {
+ return builder.stack(ThriftMux.client().withClientId(clientId));
+ } else {
+ return builder.codec(ThriftClientFramedCodec.apply(Option.apply(clientId)));
+ }
+ }
+
+ private ClientBuilder getDefaultClientBuilder(ClientConfig clientConfig) {
+ ClientBuilder builder = ClientBuilder.get()
+ .tcpConnectTimeout(Duration.fromMilliseconds(200))
+ .connectTimeout(Duration.fromMilliseconds(200))
+ .requestTimeout(Duration.fromSeconds(1));
+ if (!clientConfig.getThriftMux()) {
+ builder = builder.hostConnectionLimit(1);
+ }
+ return builder;
+ }
+
+ @SuppressWarnings("unchecked")
+ private ClientBuilder setDefaultSettings(ClientBuilder builder) {
+ return builder.name(clientName)
+ .failFast(false)
+ .noFailureAccrual()
+ // disable retries on finagle client builder, as there is only one host per finagle client
+ // we should throw exception immediately on first failure, so DL client could quickly detect
+ // failures and retry other proxies.
+ .retries(1)
+ .keepAlive(true);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public ProxyClient build(SocketAddress address) {
+ Service<ThriftClientRequest, byte[]> client =
+ ClientBuilder.safeBuildFactory(
+ clientBuilder
+ .hosts((InetSocketAddress) address)
+ .reportTo(clientStats.getFinagleStatsReceiver(address))
+ ).toService();
+ DistributedLogService.ServiceIface service =
+ new DistributedLogService.ServiceToClient(client, new TBinaryProtocol.Factory());
+ return new ProxyClient(address, client, service);
+ }
+
+ }
+
+ private final SocketAddress address;
+ private final Service<ThriftClientRequest, byte[]> client;
+ private final DistributedLogService.ServiceIface service;
+
+ protected ProxyClient(SocketAddress address,
+ Service<ThriftClientRequest, byte[]> client,
+ DistributedLogService.ServiceIface service) {
+ this.address = address;
+ this.client = client;
+ this.service = service;
+ }
+
+ public SocketAddress getAddress() {
+ return address;
+ }
+
+ public Service<ThriftClientRequest, byte[]> getClient() {
+ return client;
+ }
+
+ public DistributedLogService.ServiceIface getService() {
+ return service;
+ }
+
+ public Future<BoxedUnit> close() {
+ return client.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java
new file mode 100644
index 0000000..17b70be
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java
@@ -0,0 +1,362 @@
+/**
+ * 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 org.apache.distributedlog.client.proxy;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.ImmutableMap;
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.stats.ClientStats;
+import org.apache.distributedlog.client.stats.OpStats;
+import org.apache.distributedlog.thrift.service.ClientInfo;
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import com.twitter.util.FutureEventListener;
+import java.net.SocketAddress;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Manager manages clients (channels) to proxies.
+ */
+public class ProxyClientManager implements TimerTask {
+
+ private static final Logger logger = LoggerFactory.getLogger(ProxyClientManager.class);
+
+ private final ClientConfig clientConfig;
+ private final ProxyClient.Builder clientBuilder;
+ private final HashedWheelTimer timer;
+ private final HostProvider hostProvider;
+ private volatile Timeout periodicHandshakeTask;
+ private final ConcurrentHashMap<SocketAddress, ProxyClient> address2Services =
+ new ConcurrentHashMap<SocketAddress, ProxyClient>();
+ private final CopyOnWriteArraySet<ProxyListener> proxyListeners =
+ new CopyOnWriteArraySet<ProxyListener>();
+ private volatile boolean closed = false;
+ private volatile boolean periodicHandshakeEnabled = true;
+ private final Stopwatch lastOwnershipSyncStopwatch;
+
+ private final OpStats handshakeStats;
+
+ public ProxyClientManager(ClientConfig clientConfig,
+ ProxyClient.Builder clientBuilder,
+ HashedWheelTimer timer,
+ HostProvider hostProvider,
+ ClientStats clientStats) {
+ this.clientConfig = clientConfig;
+ this.clientBuilder = clientBuilder;
+ this.timer = timer;
+ this.hostProvider = hostProvider;
+ this.handshakeStats = clientStats.getOpStats("handshake");
+ scheduleHandshake();
+ this.lastOwnershipSyncStopwatch = Stopwatch.createStarted();
+ }
+
+ private void scheduleHandshake() {
+ if (clientConfig.getPeriodicHandshakeIntervalMs() > 0) {
+ periodicHandshakeTask = timer.newTimeout(this,
+ clientConfig.getPeriodicHandshakeIntervalMs(), TimeUnit.MILLISECONDS);
+ }
+ }
+
+ void setPeriodicHandshakeEnabled(boolean enabled) {
+ this.periodicHandshakeEnabled = enabled;
+ }
+
+ @Override
+ public void run(Timeout timeout) throws Exception {
+ if (timeout.isCancelled() || closed) {
+ return;
+ }
+ if (periodicHandshakeEnabled) {
+ final boolean syncOwnerships = lastOwnershipSyncStopwatch.elapsed(TimeUnit.MILLISECONDS)
+ >= clientConfig.getPeriodicOwnershipSyncIntervalMs();
+
+ final Set<SocketAddress> hostsSnapshot = hostProvider.getHosts();
+ final AtomicInteger numHosts = new AtomicInteger(hostsSnapshot.size());
+ final AtomicInteger numStreams = new AtomicInteger(0);
+ final AtomicInteger numSuccesses = new AtomicInteger(0);
+ final AtomicInteger numFailures = new AtomicInteger(0);
+ final ConcurrentMap<SocketAddress, Integer> streamDistributions =
+ new ConcurrentHashMap<SocketAddress, Integer>();
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ for (SocketAddress host : hostsSnapshot) {
+ final SocketAddress address = host;
+ final ProxyClient client = getClient(address);
+ handshake(address, client, new FutureEventListener<ServerInfo>() {
+ @Override
+ public void onSuccess(ServerInfo serverInfo) {
+ numStreams.addAndGet(serverInfo.getOwnershipsSize());
+ numSuccesses.incrementAndGet();
+ notifyHandshakeSuccess(address, client, serverInfo, false, stopwatch);
+ if (clientConfig.isHandshakeTracingEnabled()) {
+ streamDistributions.putIfAbsent(address, serverInfo.getOwnershipsSize());
+ }
+ complete();
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ numFailures.incrementAndGet();
+ notifyHandshakeFailure(address, client, cause, stopwatch);
+ complete();
+ }
+
+ private void complete() {
+ if (0 == numHosts.decrementAndGet()) {
+ if (syncOwnerships) {
+ logger.info("Periodic handshaked with {} hosts : {} streams returned,"
+ + " {} hosts succeeded, {} hosts failed",
+ new Object[] {
+ hostsSnapshot.size(),
+ numStreams.get(),
+ numSuccesses.get(),
+ numFailures.get()});
+ if (clientConfig.isHandshakeTracingEnabled()) {
+ logger.info("Periodic handshaked stream distribution : {}", streamDistributions);
+ }
+ }
+ }
+ }
+ }, false, syncOwnerships);
+ }
+
+ if (syncOwnerships) {
+ lastOwnershipSyncStopwatch.reset().start();
+ }
+ }
+ scheduleHandshake();
+ }
+
+ /**
+ * Register a proxy <code>listener</code> on proxy related changes.
+ *
+ * @param listener
+ * proxy listener
+ */
+ public void registerProxyListener(ProxyListener listener) {
+ proxyListeners.add(listener);
+ }
+
+ private void notifyHandshakeSuccess(SocketAddress address,
+ ProxyClient client,
+ ServerInfo serverInfo,
+ boolean logging,
+ Stopwatch stopwatch) {
+ if (logging) {
+ if (null != serverInfo && serverInfo.isSetOwnerships()) {
+ logger.info("Handshaked with {} : {} ownerships returned.",
+ address, serverInfo.getOwnerships().size());
+ } else {
+ logger.info("Handshaked with {} : no ownerships returned", address);
+ }
+ }
+ handshakeStats.completeRequest(address, stopwatch.elapsed(TimeUnit.MICROSECONDS), 1);
+ for (ProxyListener listener : proxyListeners) {
+ listener.onHandshakeSuccess(address, client, serverInfo);
+ }
+ }
+
+ private void notifyHandshakeFailure(SocketAddress address,
+ ProxyClient client,
+ Throwable cause,
+ Stopwatch stopwatch) {
+ handshakeStats.failRequest(address, stopwatch.elapsed(TimeUnit.MICROSECONDS), 1);
+ for (ProxyListener listener : proxyListeners) {
+ listener.onHandshakeFailure(address, client, cause);
+ }
+ }
+
+ /**
+ * Retrieve a client to proxy <code>address</code>.
+ *
+ * @param address
+ * proxy address
+ * @return proxy client
+ */
+ public ProxyClient getClient(final SocketAddress address) {
+ ProxyClient sc = address2Services.get(address);
+ if (null != sc) {
+ return sc;
+ }
+ return createClient(address);
+ }
+
+ /**
+ * Remove the client to proxy <code>address</code>.
+ *
+ * @param address
+ * proxy address
+ */
+ public void removeClient(SocketAddress address) {
+ ProxyClient sc = address2Services.remove(address);
+ if (null != sc) {
+ logger.info("Removed host {}.", address);
+ sc.close();
+ }
+ }
+
+ /**
+ * Remove the client <code>sc</code> to proxy <code>address</code>.
+ *
+ * @param address
+ * proxy address
+ * @param sc
+ * proxy client
+ */
+ public void removeClient(SocketAddress address, ProxyClient sc) {
+ if (address2Services.remove(address, sc)) {
+ logger.info("Remove client {} to host {}.", sc, address);
+ sc.close();
+ }
+ }
+
+ /**
+ * Create a client to proxy <code>address</code>.
+ *
+ * @param address
+ * proxy address
+ * @return proxy client
+ */
+ public ProxyClient createClient(final SocketAddress address) {
+ final ProxyClient sc = clientBuilder.build(address);
+ ProxyClient oldSC = address2Services.putIfAbsent(address, sc);
+ if (null != oldSC) {
+ sc.close();
+ return oldSC;
+ } else {
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ FutureEventListener<ServerInfo> listener = new FutureEventListener<ServerInfo>() {
+ @Override
+ public void onSuccess(ServerInfo serverInfo) {
+ notifyHandshakeSuccess(address, sc, serverInfo, true, stopwatch);
+ }
+ @Override
+ public void onFailure(Throwable cause) {
+ notifyHandshakeFailure(address, sc, cause, stopwatch);
+ }
+ };
+ // send a ping messaging after creating connections.
+ handshake(address, sc, listener, true, true);
+ return sc;
+ }
+ }
+
+ /**
+ * Handshake with a given proxy.
+ *
+ * @param address
+ * proxy address
+ * @param sc
+ * proxy client
+ * @param listener
+ * listener on handshake result
+ */
+ private void handshake(SocketAddress address,
+ ProxyClient sc,
+ FutureEventListener<ServerInfo> listener,
+ boolean logging,
+ boolean getOwnerships) {
+ if (clientConfig.getHandshakeWithClientInfo()) {
+ ClientInfo clientInfo = new ClientInfo();
+ clientInfo.setGetOwnerships(getOwnerships);
+ clientInfo.setStreamNameRegex(clientConfig.getStreamNameRegex());
+ if (logging) {
+ logger.info("Handshaking with {} : {}", address, clientInfo);
+ }
+ sc.getService().handshakeWithClientInfo(clientInfo)
+ .addEventListener(listener);
+ } else {
+ if (logging) {
+ logger.info("Handshaking with {}", address);
+ }
+ sc.getService().handshake().addEventListener(listener);
+ }
+ }
+
+ /**
+ * Handshake with all proxies.
+ *
+ * <p>NOTE: this is a synchronous call.
+ */
+ public void handshake() {
+ Set<SocketAddress> hostsSnapshot = hostProvider.getHosts();
+ logger.info("Handshaking with {} hosts.", hostsSnapshot.size());
+ final CountDownLatch latch = new CountDownLatch(hostsSnapshot.size());
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ for (SocketAddress host: hostsSnapshot) {
+ final SocketAddress address = host;
+ final ProxyClient client = getClient(address);
+ handshake(address, client, new FutureEventListener<ServerInfo>() {
+ @Override
+ public void onSuccess(ServerInfo serverInfo) {
+ notifyHandshakeSuccess(address, client, serverInfo, true, stopwatch);
+ latch.countDown();
+ }
+ @Override
+ public void onFailure(Throwable cause) {
+ notifyHandshakeFailure(address, client, cause, stopwatch);
+ latch.countDown();
+ }
+ }, true, true);
+ }
+ try {
+ latch.await(1, TimeUnit.MINUTES);
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted on handshaking with servers : ", e);
+ }
+ }
+
+ /**
+ * Return number of proxies managed by client manager.
+ *
+ * @return number of proxies managed by client manager.
+ */
+ public int getNumProxies() {
+ return address2Services.size();
+ }
+
+ /**
+ * Return all clients.
+ *
+ * @return all clients.
+ */
+ public Map<SocketAddress, ProxyClient> getAllClients() {
+ return ImmutableMap.copyOf(address2Services);
+ }
+
+ public void close() {
+ closed = true;
+ Timeout task = periodicHandshakeTask;
+ if (null != task) {
+ task.cancel();
+ }
+ for (ProxyClient sc : address2Services.values()) {
+ sc.close();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java
new file mode 100644
index 0000000..0a6b076
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java
@@ -0,0 +1,50 @@
+/**
+ * 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 org.apache.distributedlog.client.proxy;
+
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import java.net.SocketAddress;
+
+/**
+ * Listener on server changes.
+ */
+public interface ProxyListener {
+ /**
+ * When a proxy's server info changed, it would be notified.
+ *
+ * @param address
+ * proxy address
+ * @param client
+ * proxy client that executes handshaking
+ * @param serverInfo
+ * proxy's server info
+ */
+ void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo);
+
+ /**
+ * Failed to handshake with a proxy.
+ *
+ * @param address
+ * proxy address
+ * @param client
+ * proxy client
+ * @param cause
+ * failure reason
+ */
+ void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause);
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java
new file mode 100644
index 0000000..4161afb
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Clients that interact with individual proxies.
+ */
+package org.apache.distributedlog.client.proxy;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java
new file mode 100644
index 0000000..2ac5be3
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.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 org.apache.distributedlog.client.resolver;
+
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Default implementation of {@link RegionResolver}.
+ */
+public class DefaultRegionResolver implements RegionResolver {
+
+ private static final String DEFAULT_REGION = "default-region";
+
+ private final Map<SocketAddress, String> regionOverrides =
+ new HashMap<SocketAddress, String>();
+ private final ConcurrentMap<SocketAddress, String> regionMap =
+ new ConcurrentHashMap<SocketAddress, String>();
+
+ public DefaultRegionResolver() {
+ }
+
+ public DefaultRegionResolver(Map<SocketAddress, String> regionOverrides) {
+ this.regionOverrides.putAll(regionOverrides);
+ }
+
+ @Override
+ public String resolveRegion(SocketAddress address) {
+ String region = regionMap.get(address);
+ if (null == region) {
+ region = doResolveRegion(address);
+ regionMap.put(address, region);
+ }
+ return region;
+ }
+
+ private String doResolveRegion(SocketAddress address) {
+ String region = regionOverrides.get(address);
+ if (null != region) {
+ return region;
+ }
+
+ String domainName;
+ if (address instanceof InetSocketAddress) {
+ InetSocketAddress iAddr = (InetSocketAddress) address;
+ domainName = iAddr.getHostName();
+ } else {
+ domainName = address.toString();
+ }
+ String[] parts = domainName.split("\\.");
+ if (parts.length <= 0) {
+ return DEFAULT_REGION;
+ }
+ String hostName = parts[0];
+ String[] labels = hostName.split("-");
+ if (labels.length != 4) {
+ return DEFAULT_REGION;
+ }
+ return labels[0];
+ }
+
+ @Override
+ public void removeCachedHost(SocketAddress address) {
+ regionMap.remove(address);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java
new file mode 100644
index 0000000..023799c
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java
@@ -0,0 +1,43 @@
+/**
+ * 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 org.apache.distributedlog.client.resolver;
+
+import java.net.SocketAddress;
+
+/**
+ * Resolve address to region.
+ */
+public interface RegionResolver {
+
+ /**
+ * Resolve address to region.
+ *
+ * @param address
+ * socket address
+ * @return region
+ */
+ String resolveRegion(SocketAddress address);
+
+ /**
+ * Remove cached host.
+ *
+ * @param address
+ * socket address.
+ */
+ void removeCachedHost(SocketAddress address);
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java
new file mode 100644
index 0000000..81cda2f
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Resolver to resolve network addresses.
+ */
+package org.apache.distributedlog.client.resolver;
[48/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Worker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Worker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Worker.java
new file mode 100644
index 0000000..a948092
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Worker.java
@@ -0,0 +1,26 @@
+/**
+ * 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 org.apache.distributedlog.benchmark;
+
+import java.io.Closeable;
+
+/**
+ * Worker to run benchmark.
+ */
+public interface Worker extends Closeable, Runnable {
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
new file mode 100644
index 0000000..9e96765
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
@@ -0,0 +1,387 @@
+/**
+ * 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 org.apache.distributedlog.benchmark;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.benchmark.utils.ShiftableRateLimiter;
+import org.apache.distributedlog.client.DistributedLogMultiStreamWriter;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import org.apache.distributedlog.util.SchedulerUtils;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration$;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Benchmark for distributedlog proxy client.
+ */
+public class WriterWorker implements Worker {
+
+ static final Logger LOG = LoggerFactory.getLogger(WriterWorker.class);
+
+ final String streamPrefix;
+ final int startStreamId;
+ final int endStreamId;
+ final int writeConcurrency;
+ final int messageSizeBytes;
+ final int hostConnectionCoreSize;
+ final int hostConnectionLimit;
+ final ExecutorService executorService;
+ final ShiftableRateLimiter rateLimiter;
+ final URI dlUri;
+ final DLZkServerSet[] serverSets;
+ final List<String> finagleNames;
+ final Random random;
+ final List<String> streamNames;
+ final int numStreams;
+ final int batchSize;
+ final boolean thriftmux;
+ final boolean handshakeWithClientInfo;
+ final int sendBufferSize;
+ final int recvBufferSize;
+ final boolean enableBatching;
+ final int batchBufferSize;
+ final int batchFlushIntervalMicros;
+ private final String routingServiceFinagleName;
+
+ volatile boolean running = true;
+
+ final StatsReceiver statsReceiver;
+ final StatsLogger statsLogger;
+ final OpStatsLogger requestStat;
+ final StatsLogger exceptionsLogger;
+ final StatsLogger dlErrorCodeLogger;
+
+ // callback thread
+ final ExecutorService executor;
+
+ public WriterWorker(String streamPrefix,
+ URI uri,
+ int startStreamId,
+ int endStreamId,
+ ShiftableRateLimiter rateLimiter,
+ int writeConcurrency,
+ int messageSizeBytes,
+ int batchSize,
+ int hostConnectionCoreSize,
+ int hostConnectionLimit,
+ List<String> serverSetPaths,
+ List<String> finagleNames,
+ StatsReceiver statsReceiver,
+ StatsLogger statsLogger,
+ boolean thriftmux,
+ boolean handshakeWithClientInfo,
+ int sendBufferSize,
+ int recvBufferSize,
+ boolean enableBatching,
+ int batchBufferSize,
+ int batchFlushIntervalMicros,
+ String routingServiceFinagleName) {
+ checkArgument(startStreamId <= endStreamId);
+ checkArgument(!finagleNames.isEmpty() || !serverSetPaths.isEmpty());
+ this.streamPrefix = streamPrefix;
+ this.dlUri = uri;
+ this.startStreamId = startStreamId;
+ this.endStreamId = endStreamId;
+ this.rateLimiter = rateLimiter;
+ this.writeConcurrency = writeConcurrency;
+ this.messageSizeBytes = messageSizeBytes;
+ this.statsReceiver = statsReceiver;
+ this.statsLogger = statsLogger;
+ this.requestStat = this.statsLogger.getOpStatsLogger("requests");
+ this.exceptionsLogger = statsLogger.scope("exceptions");
+ this.dlErrorCodeLogger = statsLogger.scope("dl_error_code");
+ this.executorService = Executors.newCachedThreadPool();
+ this.random = new Random(System.currentTimeMillis());
+ this.batchSize = batchSize;
+ this.hostConnectionCoreSize = hostConnectionCoreSize;
+ this.hostConnectionLimit = hostConnectionLimit;
+ this.thriftmux = thriftmux;
+ this.handshakeWithClientInfo = handshakeWithClientInfo;
+ this.sendBufferSize = sendBufferSize;
+ this.recvBufferSize = recvBufferSize;
+ this.enableBatching = enableBatching;
+ this.batchBufferSize = batchBufferSize;
+ this.batchFlushIntervalMicros = batchFlushIntervalMicros;
+ this.finagleNames = finagleNames;
+ this.serverSets = createServerSets(serverSetPaths);
+ this.executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
+ this.routingServiceFinagleName = routingServiceFinagleName;
+
+ // Streams
+ streamNames = new ArrayList<String>(endStreamId - startStreamId);
+ for (int i = startStreamId; i < endStreamId; i++) {
+ streamNames.add(String.format("%s_%d", streamPrefix, i));
+ }
+ numStreams = streamNames.size();
+ LOG.info("Writing to {} streams : {}", numStreams, streamNames);
+ }
+
+ protected DLZkServerSet[] createServerSets(List<String> serverSetPaths) {
+ DLZkServerSet[] serverSets = new DLZkServerSet[serverSetPaths.size()];
+ for (int i = 0; i < serverSets.length; i++) {
+ String serverSetPath = serverSetPaths.get(i);
+ serverSets[i] = DLZkServerSet.of(URI.create(serverSetPath), 60000);
+ }
+ return serverSets;
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.running = false;
+ SchedulerUtils.shutdownScheduler(this.executorService, 2, TimeUnit.MINUTES);
+ for (DLZkServerSet serverSet: serverSets) {
+ serverSet.close();
+ }
+ }
+
+ private DistributedLogClient buildDlogClient() {
+ ClientBuilder clientBuilder = ClientBuilder.get()
+ .hostConnectionLimit(hostConnectionLimit)
+ .hostConnectionCoresize(hostConnectionCoreSize)
+ .tcpConnectTimeout(Duration$.MODULE$.fromMilliseconds(200))
+ .connectTimeout(Duration$.MODULE$.fromMilliseconds(200))
+ .requestTimeout(Duration$.MODULE$.fromSeconds(10))
+ .sendBufferSize(sendBufferSize)
+ .recvBufferSize(recvBufferSize);
+
+ ClientId clientId = ClientId$.MODULE$.apply("dlog_loadtest_writer");
+
+ DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
+ .clientId(clientId)
+ .clientBuilder(clientBuilder)
+ .thriftmux(thriftmux)
+ .redirectBackoffStartMs(100)
+ .redirectBackoffMaxMs(500)
+ .requestTimeoutMs(10000)
+ .statsReceiver(statsReceiver)
+ .streamNameRegex("^" + streamPrefix + "_[0-9]+$")
+ .handshakeWithClientInfo(handshakeWithClientInfo)
+ .periodicHandshakeIntervalMs(TimeUnit.SECONDS.toMillis(30))
+ .periodicOwnershipSyncIntervalMs(TimeUnit.MINUTES.toMillis(5))
+ .periodicDumpOwnershipCache(true)
+ .handshakeTracing(true)
+ .serverRoutingServiceFinagleNameStr(routingServiceFinagleName)
+ .name("writer");
+
+ if (!finagleNames.isEmpty()) {
+ String local = finagleNames.get(0);
+ String[] remotes = new String[finagleNames.size() - 1];
+ finagleNames.subList(1, finagleNames.size()).toArray(remotes);
+
+ builder = builder.finagleNameStrs(local, remotes);
+ } else if (serverSets.length != 0){
+ ServerSet local = serverSets[0].getServerSet();
+ ServerSet[] remotes = new ServerSet[serverSets.length - 1];
+ for (int i = 1; i < serverSets.length; i++) {
+ remotes[i - 1] = serverSets[i].getServerSet();
+ }
+ builder = builder.serverSets(local, remotes);
+ } else {
+ builder = builder.uri(dlUri);
+ }
+
+ return builder.build();
+ }
+
+ ByteBuffer buildBuffer(long requestMillis, int messageSizeBytes) {
+ ByteBuffer data;
+ try {
+ data = ByteBuffer.wrap(Utils.generateMessage(requestMillis, messageSizeBytes));
+ return data;
+ } catch (TException e) {
+ LOG.error("Error generating message : ", e);
+ return null;
+ }
+ }
+
+ List<ByteBuffer> buildBufferList(int batchSize, long requestMillis, int messageSizeBytes) {
+ ArrayList<ByteBuffer> bufferList = new ArrayList<ByteBuffer>(batchSize);
+ for (int i = 0; i < batchSize; i++) {
+ ByteBuffer buf = buildBuffer(requestMillis, messageSizeBytes);
+ if (null == buf) {
+ return null;
+ }
+ bufferList.add(buf);
+ }
+ return bufferList;
+ }
+
+ class TimedRequestHandler implements FutureEventListener<DLSN>, Runnable {
+ final String streamName;
+ final long requestMillis;
+ DLSN dlsn = null;
+ Throwable cause = null;
+
+ TimedRequestHandler(String streamName,
+ long requestMillis) {
+ this.streamName = streamName;
+ this.requestMillis = requestMillis;
+ }
+ @Override
+ public void onSuccess(DLSN value) {
+ dlsn = value;
+ executor.submit(this);
+ }
+ @Override
+ public void onFailure(Throwable cause) {
+ this.cause = cause;
+ executor.submit(this);
+ }
+
+ @Override
+ public void run() {
+ if (null != dlsn) {
+ requestStat.registerSuccessfulEvent(System.currentTimeMillis() - requestMillis);
+ } else {
+ LOG.error("Failed to publish to {} : ", streamName, cause);
+ requestStat.registerFailedEvent(System.currentTimeMillis() - requestMillis);
+ exceptionsLogger.getCounter(cause.getClass().getName()).inc();
+ if (cause instanceof DLException) {
+ DLException dle = (DLException) cause;
+ dlErrorCodeLogger.getCounter(dle.getCode().toString()).inc();
+ }
+ }
+ }
+ }
+
+ class Writer implements Runnable {
+
+ final int idx;
+ final DistributedLogClient dlc;
+ DistributedLogMultiStreamWriter writer = null;
+ final ShiftableRateLimiter limiter;
+
+ Writer(int idx) {
+ this.idx = idx;
+ this.dlc = buildDlogClient();
+ if (enableBatching) {
+ writer = DistributedLogMultiStreamWriter.newBuilder()
+ .client(this.dlc)
+ .streams(streamNames)
+ .compressionCodec(CompressionCodec.Type.NONE)
+ .flushIntervalMicros(batchFlushIntervalMicros)
+ .bufferSize(batchBufferSize)
+ .firstSpeculativeTimeoutMs(9000)
+ .maxSpeculativeTimeoutMs(9000)
+ .requestTimeoutMs(10000)
+ .speculativeBackoffMultiplier(2)
+ .build();
+ }
+ this.limiter = rateLimiter.duplicate();
+ }
+
+ @Override
+ public void run() {
+ LOG.info("Started writer {}.", idx);
+ while (running) {
+ this.limiter.getLimiter().acquire();
+ final String streamName = streamNames.get(random.nextInt(numStreams));
+ final long requestMillis = System.currentTimeMillis();
+ final ByteBuffer data = buildBuffer(requestMillis, messageSizeBytes);
+ if (null == data) {
+ break;
+ }
+ if (null != writer) {
+ writer.write(data).addEventListener(
+ new TimedRequestHandler(streamName, requestMillis));
+ } else {
+ dlc.write(streamName, data).addEventListener(
+ new TimedRequestHandler(streamName, requestMillis));
+ }
+ }
+ if (null != writer) {
+ writer.close();
+ }
+ dlc.close();
+ }
+ }
+
+ class BulkWriter implements Runnable {
+
+ final int idx;
+ final DistributedLogClient dlc;
+
+ BulkWriter(int idx) {
+ this.idx = idx;
+ this.dlc = buildDlogClient();
+ }
+
+ @Override
+ public void run() {
+ LOG.info("Started writer {}.", idx);
+ while (running) {
+ rateLimiter.getLimiter().acquire(batchSize);
+ String streamName = streamNames.get(random.nextInt(numStreams));
+ final long requestMillis = System.currentTimeMillis();
+ final List<ByteBuffer> data = buildBufferList(batchSize, requestMillis, messageSizeBytes);
+ if (null == data) {
+ break;
+ }
+ List<Future<DLSN>> results = dlc.writeBulk(streamName, data);
+ for (Future<DLSN> result : results) {
+ result.addEventListener(new TimedRequestHandler(streamName, requestMillis));
+ }
+ }
+ dlc.close();
+ }
+ }
+
+ @Override
+ public void run() {
+ LOG.info("Starting writer (concurrency = {}, prefix = {}, batchSize = {})",
+ new Object[] { writeConcurrency, streamPrefix, batchSize });
+ try {
+ for (int i = 0; i < writeConcurrency; i++) {
+ Runnable writer = null;
+ if (batchSize > 0) {
+ writer = new BulkWriter(i);
+ } else {
+ writer = new Writer(i);
+ }
+ executorService.submit(writer);
+ }
+ } catch (Throwable t) {
+ LOG.error("Unhandled exception caught", t);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/package-info.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/package-info.java
new file mode 100644
index 0000000..7e87644
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Benchmarks for distributedlog.
+ */
+package org.apache.distributedlog.benchmark;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AbstractReaderBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
new file mode 100644
index 0000000..a1f1f9f
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
@@ -0,0 +1,70 @@
+/**
+ * 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 org.apache.distributedlog.benchmark.stream;
+
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.commons.cli.CommandLine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class AbstractReaderBenchmark extends StreamBenchmark {
+
+ private static final Logger logger = LoggerFactory.getLogger(SyncReaderBenchmark.class);
+
+ protected ReadMode readMode = ReadMode.LATEST;
+ protected long fromTxId = DistributedLogConstants.INVALID_TXID;
+ protected long rewindMs = 0L;
+ protected int batchSize = 1;
+
+ protected AbstractReaderBenchmark() {
+ options.addOption("t", "tx-id", true,
+ "Transaction ID to start read from when reading in mode 'position'");
+ options.addOption("r", "rewind", true,
+ "Time to rewind back to read from when reading in mode 'rewind' (in milliseconds)");
+ options.addOption("m", "mode", true,
+ "Read Mode : [oldest, latest, rewind, position]");
+ options.addOption("b", "batch-size", true, "Read batch size");
+ }
+
+ @Override
+ protected void parseCommandLine(CommandLine cmdline) {
+ if (cmdline.hasOption("m")) {
+ String mode = cmdline.getOptionValue("m");
+ try {
+ readMode = ReadMode.valueOf(mode.toUpperCase());
+ } catch (IllegalArgumentException iae) {
+ logger.error("Invalid read mode {}.", mode);
+ printUsage();
+ System.exit(0);
+ }
+ } else {
+ printUsage();
+ System.exit(0);
+ }
+ if (cmdline.hasOption("t")) {
+ fromTxId = Long.parseLong(cmdline.getOptionValue("t"));
+ }
+ if (cmdline.hasOption("r")) {
+ rewindMs = Long.parseLong(cmdline.getOptionValue("r"));
+ }
+ if (cmdline.hasOption("b")) {
+ batchSize = Integer.parseInt(cmdline.getOptionValue("b"));
+ }
+ logger.info("Start reading from transaction id {}, rewind {} ms.", fromTxId, rewindMs);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
new file mode 100644
index 0000000..4930b8a
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
@@ -0,0 +1,158 @@
+/**
+ * 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 org.apache.distributedlog.benchmark.stream;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.AsyncLogReader;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.util.FutureUtils;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Benchmark on {@link org.apache.distributedlog.AsyncLogReader} reading from a stream.
+ */
+public class AsyncReaderBenchmark extends AbstractReaderBenchmark {
+
+ private static final Logger logger = LoggerFactory.getLogger(AsyncReaderBenchmark.class);
+
+ @Override
+ protected void benchmark(DistributedLogNamespace namespace, String logName, StatsLogger statsLogger) {
+ DistributedLogManager dlm = null;
+ while (null == dlm) {
+ try {
+ dlm = namespace.openLog(streamName);
+ } catch (IOException ioe) {
+ logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
+ }
+ if (null == dlm) {
+ try {
+ TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
+ streamName, e);
+ }
+ }
+ }
+ logger.info("Created dlm for stream {}.", streamName);
+
+ // Stats
+ OpStatsLogger openReaderStats = statsLogger.getOpStatsLogger("open_reader");
+ OpStatsLogger blockingReadStats = statsLogger.getOpStatsLogger("blocking_read");
+ Counter readCounter = statsLogger.getCounter("reads");
+
+ AsyncLogReader reader = null;
+ DLSN lastDLSN = null;
+ Long lastTxId = null;
+ while (null == reader) {
+ // initialize the last txid
+ if (null == lastTxId) {
+ switch (readMode) {
+ case OLDEST:
+ lastTxId = 0L;
+ lastDLSN = DLSN.InitialDLSN;
+ break;
+ case LATEST:
+ lastTxId = Long.MAX_VALUE;
+ try {
+ lastDLSN = dlm.getLastDLSN();
+ } catch (IOException ioe) {
+ continue;
+ }
+ break;
+ case REWIND:
+ lastTxId = System.currentTimeMillis() - rewindMs;
+ lastDLSN = null;
+ break;
+ case POSITION:
+ lastTxId = fromTxId;
+ lastDLSN = null;
+ break;
+ default:
+ logger.warn("Unsupported mode {}", readMode);
+ printUsage();
+ System.exit(0);
+ break;
+ }
+ logger.info("Reading from transaction id = {}, dlsn = {}", lastTxId, lastDLSN);
+ }
+ // Open the reader
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ try {
+ if (null == lastDLSN) {
+ reader = FutureUtils.result(dlm.openAsyncLogReader(lastTxId));
+ } else {
+ reader = FutureUtils.result(dlm.openAsyncLogReader(lastDLSN));
+ }
+ long elapsedMs = stopwatch.elapsed(TimeUnit.MICROSECONDS);
+ openReaderStats.registerSuccessfulEvent(elapsedMs);
+ logger.info("It took {} ms to position the reader to transaction id = {}, dlsn = {}",
+ lastTxId, lastDLSN);
+ } catch (IOException ioe) {
+ openReaderStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+ logger.warn("Failed to create reader for stream {} reading from tx id = {}, dlsn = {}.",
+ new Object[] { streamName, lastTxId, lastDLSN });
+ }
+ if (null == reader) {
+ try {
+ TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted from sleep after reader was reassigned null for stream {} : ",
+ streamName, e);
+ }
+ continue;
+ }
+ List<LogRecordWithDLSN> records;
+ stopwatch = Stopwatch.createUnstarted();
+ while (true) {
+ try {
+ stopwatch.start();
+ records = FutureUtils.result(reader.readBulk(batchSize));
+ long elapsedMicros = stopwatch.stop().elapsed(TimeUnit.MICROSECONDS);
+ blockingReadStats.registerSuccessfulEvent(elapsedMicros);
+ if (!records.isEmpty()) {
+ readCounter.add(records.size());
+ LogRecordWithDLSN lastRecord = records.get(records.size() - 1);
+ lastTxId = lastRecord.getTransactionId();
+ lastDLSN = lastRecord.getDlsn();
+ }
+ stopwatch.reset();
+ } catch (IOException e) {
+ logger.warn("Encountered reading record from stream {} : ", streamName, e);
+ reader = null;
+ break;
+ }
+ }
+ try {
+ TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted from sleep while creating reader for stream {} : ",
+ streamName, e);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerBatchReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerBatchReader.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerBatchReader.java
new file mode 100644
index 0000000..b115192
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerBatchReader.java
@@ -0,0 +1,82 @@
+/**
+ * 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 org.apache.distributedlog.benchmark.stream;
+
+import java.util.Enumeration;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Read ledgers in batches.
+ */
+public class LedgerBatchReader implements Runnable {
+
+ private static final Logger logger = LoggerFactory.getLogger(LedgerBatchReader.class);
+
+ private final LedgerHandle lh;
+ private final ReadEntryListener readEntryListener;
+ private final int batchSize;
+
+ public LedgerBatchReader(LedgerHandle lh,
+ ReadEntryListener readEntryListener,
+ int batchSize) {
+ this.lh = lh;
+ this.batchSize = batchSize;
+ this.readEntryListener = readEntryListener;
+ }
+
+ @Override
+ public void run() {
+ long lac = lh.getLastAddConfirmed();
+
+ long entryId = 0L;
+
+ while (entryId <= lac) {
+ long startEntryId = entryId;
+ long endEntryId = Math.min(startEntryId + batchSize - 1, lac);
+
+ Enumeration<LedgerEntry> entries = null;
+ while (null == entries) {
+ try {
+ entries = lh.readEntries(startEntryId, endEntryId);
+ } catch (BKException bke) {
+ logger.error("Encountered exceptions on reading [ {} - {} ] ",
+ new Object[] { startEntryId, endEntryId, bke });
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ break;
+ }
+ }
+ if (null == entries) {
+ break;
+ }
+
+ while (entries.hasMoreElements()) {
+ LedgerEntry entry = entries.nextElement();
+ readEntryListener.onEntryComplete(BKException.Code.OK, lh, entry, null);
+ }
+
+ entryId = endEntryId + 1;
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java
new file mode 100644
index 0000000..489e5af
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java
@@ -0,0 +1,151 @@
+/**
+ * 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 org.apache.distributedlog.benchmark.stream;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.BookKeeperClientBuilder;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Benchmark ledger reading.
+ */
+public class LedgerReadBenchmark extends AbstractReaderBenchmark {
+
+ private static final Logger logger = LoggerFactory.getLogger(AsyncReaderBenchmark.class);
+
+ @Override
+ protected void benchmark(DistributedLogNamespace namespace, String logName, StatsLogger statsLogger) {
+ DistributedLogManager dlm = null;
+ while (null == dlm) {
+ try {
+ dlm = namespace.openLog(streamName);
+ } catch (IOException ioe) {
+ logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
+ }
+ if (null == dlm) {
+ try {
+ TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
+ streamName, e);
+ }
+ }
+ }
+ logger.info("Created dlm for stream {}.", streamName);
+
+ List<LogSegmentMetadata> segments = null;
+ while (null == segments) {
+ try {
+ segments = dlm.getLogSegments();
+ } catch (IOException ioe) {
+ logger.warn("Failed to get log segments for stream {} : ", streamName, ioe);
+ }
+ if (null == segments) {
+ try {
+ TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted from sleep while geting log segments for stream {} : ",
+ streamName, e);
+ }
+ }
+ }
+
+ final Counter readCounter = statsLogger.getCounter("reads");
+
+ logger.info("Reading from log segments : {}", segments);
+
+ ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
+ .uri(uri)
+ .name("benchmark-zkc")
+ .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+ .zkAclId(null)
+ .build();
+ BKDLConfig bkdlConfig;
+ try {
+ bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+ } catch (IOException e) {
+ return;
+ }
+
+ BookKeeper bk;
+ try {
+ bk = BookKeeperClientBuilder.newBuilder()
+ .name("benchmark-bkc")
+ .dlConfig(conf)
+ .zkServers(bkdlConfig.getBkZkServersForReader())
+ .ledgersPath(bkdlConfig.getBkLedgersPath())
+ .build()
+ .get();
+ } catch (IOException e) {
+ return;
+ }
+
+ final int readConcurrency = conf.getInt("ledger_read_concurrency", 1000);
+ boolean streamRead = conf.getBoolean("ledger_stream_read", true);
+ try {
+ for (LogSegmentMetadata segment : segments) {
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ long lid = segment.getLogSegmentId();
+ LedgerHandle lh = bk.openLedgerNoRecovery(
+ lid, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
+ logger.info("It took {} ms to open log segment {}",
+ new Object[] { stopwatch.elapsed(TimeUnit.MILLISECONDS), (lh.getLastAddConfirmed() + 1), segment });
+ stopwatch.reset().start();
+ Runnable reader;
+ if (streamRead) {
+ reader = new LedgerStreamReader(lh, new BookkeeperInternalCallbacks.ReadEntryListener() {
+ @Override
+ public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object ctx) {
+ readCounter.inc();
+ }
+ }, readConcurrency);
+ } else {
+ reader = new LedgerStreamReader(lh, new BookkeeperInternalCallbacks.ReadEntryListener() {
+ @Override
+ public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object ctx) {
+ readCounter.inc();
+ }
+ }, readConcurrency);
+ }
+ reader.run();
+ logger.info("It took {} ms to complete reading {} entries from log segment {}",
+ new Object[] { stopwatch.elapsed(TimeUnit.MILLISECONDS), (lh.getLastAddConfirmed() + 1), segment });
+ }
+ } catch (Exception e) {
+ logger.error("Error on reading bk ", e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerStreamReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerStreamReader.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerStreamReader.java
new file mode 100644
index 0000000..11c3482
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerStreamReader.java
@@ -0,0 +1,131 @@
+/**
+ * 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 org.apache.distributedlog.benchmark.stream;
+
+import java.util.Enumeration;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.bookkeeper.client.AsyncCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Reading ledger in a streaming way.
+ */
+public class LedgerStreamReader implements Runnable {
+
+ private static final Logger logger = LoggerFactory.getLogger(LedgerStreamReader.class);
+
+ class PendingReadRequest implements AsyncCallback.ReadCallback {
+
+ final long entryId;
+ boolean isDone = false;
+ int rc;
+ LedgerEntry entry = null;
+
+ PendingReadRequest(long entryId) {
+ this.entryId = entryId;
+ }
+
+ void read() {
+ lh.asyncReadEntries(entryId, entryId, this, null);
+ }
+
+ void complete(ReadEntryListener listener) {
+ listener.onEntryComplete(rc, lh, entry, null);
+ }
+
+ @Override
+ public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> enumeration, Object ctx) {
+ this.rc = rc;
+ if (BKException.Code.OK == rc && enumeration.hasMoreElements()) {
+ entry = enumeration.nextElement();
+ } else {
+ entry = null;
+ }
+ isDone = true;
+ // construct a new read request
+ long nextEntry = nextReadEntry.getAndIncrement();
+ if (nextEntry <= lac) {
+ PendingReadRequest nextRead =
+ new PendingReadRequest(nextEntry);
+ pendingReads.add(nextRead);
+ nextRead.read();
+ }
+ triggerCallbacks();
+ }
+ }
+
+ private final LedgerHandle lh;
+ private final long lac;
+ private final ReadEntryListener readEntryListener;
+ private final int concurrency;
+ private final AtomicLong nextReadEntry = new AtomicLong(0);
+ private final CountDownLatch done = new CountDownLatch(1);
+ private final ConcurrentLinkedQueue<PendingReadRequest> pendingReads =
+ new ConcurrentLinkedQueue<PendingReadRequest>();
+
+ public LedgerStreamReader(LedgerHandle lh,
+ ReadEntryListener readEntryListener,
+ int concurrency) {
+ this.lh = lh;
+ this.lac = lh.getLastAddConfirmed();
+ this.readEntryListener = readEntryListener;
+ this.concurrency = concurrency;
+ for (int i = 0; i < concurrency; i++) {
+ long entryId = nextReadEntry.getAndIncrement();
+ if (entryId > lac) {
+ break;
+ }
+ PendingReadRequest request = new PendingReadRequest(entryId);
+ pendingReads.add(request);
+ request.read();
+ }
+ if (pendingReads.isEmpty()) {
+ done.countDown();
+ }
+ }
+
+ synchronized void triggerCallbacks() {
+ PendingReadRequest request;
+ while ((request = pendingReads.peek()) != null) {
+ if (!request.isDone) {
+ break;
+ }
+ pendingReads.remove();
+ request.complete(readEntryListener);
+ }
+ if (pendingReads.isEmpty()) {
+ done.countDown();
+ }
+ }
+
+ @Override
+ public void run() {
+ try {
+ done.await();
+ } catch (InterruptedException e) {
+ logger.info("Interrupted on stream reading ledger {} : ", lh.getId(), e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/ReadMode.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/ReadMode.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/ReadMode.java
new file mode 100644
index 0000000..ea5ed36
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/ReadMode.java
@@ -0,0 +1,28 @@
+/**
+ * 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 org.apache.distributedlog.benchmark.stream;
+
+/**
+ * The read mode for streaming read benchmark.
+ */
+public enum ReadMode {
+ OLDEST,
+ LATEST,
+ REWIND,
+ POSITION
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.java
new file mode 100644
index 0000000..d3083ca
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.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 org.apache.distributedlog.benchmark.stream;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import java.io.File;
+import java.net.URI;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Benchmark Streams.
+ */
+public abstract class StreamBenchmark {
+
+ private static final Logger logger = LoggerFactory.getLogger(StreamBenchmark.class);
+
+ private static final String USAGE = "StreamBenchmark <benchmark-class> [options]";
+
+ protected final Options options = new Options();
+ protected URI uri;
+ protected DistributedLogConfiguration conf;
+ protected StatsProvider statsProvider;
+ protected String streamName;
+
+ protected StreamBenchmark() {
+ options.addOption("c", "conf", true, "Configuration File");
+ options.addOption("u", "uri", true, "DistributedLog URI");
+ options.addOption("p", "stats-provider", true, "Stats Provider");
+ options.addOption("s", "stream", true, "Stream Name");
+ options.addOption("h", "help", false, "Print usage.");
+ }
+
+ protected Options getOptions() {
+ return options;
+ }
+
+ protected void printUsage() {
+ HelpFormatter hf = new HelpFormatter();
+ hf.printHelp(USAGE, options);
+ }
+
+ protected void parseCommandLine(String[] args)
+ throws Exception {
+ BasicParser parser = new BasicParser();
+ CommandLine cmdline = parser.parse(options, args);
+ if (cmdline.hasOption("h")) {
+ printUsage();
+ System.exit(0);
+ }
+ if (cmdline.hasOption("u")) {
+ this.uri = URI.create(cmdline.getOptionValue("u"));
+ } else {
+ printUsage();
+ System.exit(0);
+ }
+ this.conf = new DistributedLogConfiguration();
+ if (cmdline.hasOption("c")) {
+ String configFile = cmdline.getOptionValue("c");
+ this.conf.loadConf(new File(configFile).toURI().toURL());
+ }
+ if (cmdline.hasOption("p")) {
+ statsProvider = ReflectionUtils.newInstance(cmdline.getOptionValue("p"), StatsProvider.class);
+ } else {
+ statsProvider = new NullStatsProvider();
+ }
+ if (cmdline.hasOption("s")) {
+ this.streamName = cmdline.getOptionValue("s");
+ } else {
+ printUsage();
+ System.exit(0);
+ }
+ parseCommandLine(cmdline);
+ }
+
+ protected abstract void parseCommandLine(CommandLine cmdline);
+
+ protected void run(String[] args) throws Exception {
+ logger.info("Parsing arguments for benchmark : {}", args);
+ // parse command line
+ parseCommandLine(args);
+ statsProvider.start(conf);
+ // run the benchmark
+ StatsLogger statsLogger = statsProvider.getStatsLogger("dl");
+ DistributedLogNamespace namespace =
+ DistributedLogNamespaceBuilder.newBuilder()
+ .conf(conf)
+ .uri(uri)
+ .statsLogger(statsLogger)
+ .build();
+ try {
+ benchmark(namespace, streamName, statsProvider.getStatsLogger("benchmark"));
+ } finally {
+ namespace.close();
+ statsProvider.stop();
+ }
+ }
+
+ protected abstract void benchmark(DistributedLogNamespace namespace,
+ String logName,
+ StatsLogger statsLogger);
+
+ public static void main(String[] args) throws Exception {
+ if (args.length <= 0) {
+ System.err.println(USAGE);
+ return;
+ }
+ String benchmarkClassName = args[0];
+ StreamBenchmark benchmark = ReflectionUtils.newInstance(
+ benchmarkClassName, StreamBenchmark.class);
+ benchmark.run(args);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java
new file mode 100644
index 0000000..4abb317
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java
@@ -0,0 +1,164 @@
+/**
+ * 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 org.apache.distributedlog.benchmark.stream;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogReader;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Benchmark on {@link org.apache.distributedlog.LogReader} reading from a stream.
+ */
+public class SyncReaderBenchmark extends AbstractReaderBenchmark {
+
+ private static final Logger logger = LoggerFactory.getLogger(SyncReaderBenchmark.class);
+
+ public SyncReaderBenchmark() {}
+
+ @Override
+ protected void benchmark(DistributedLogNamespace namespace, String streamName, StatsLogger statsLogger) {
+ DistributedLogManager dlm = null;
+ while (null == dlm) {
+ try {
+ dlm = namespace.openLog(streamName);
+ } catch (IOException ioe) {
+ logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
+ }
+ if (null == dlm) {
+ try {
+ TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
+ streamName, e);
+ }
+ }
+ }
+ OpStatsLogger openReaderStats = statsLogger.getOpStatsLogger("open_reader");
+ OpStatsLogger nonBlockingReadStats = statsLogger.getOpStatsLogger("non_blocking_read");
+ OpStatsLogger blockingReadStats = statsLogger.getOpStatsLogger("blocking_read");
+ Counter nullReadCounter = statsLogger.getCounter("null_read");
+
+ logger.info("Created dlm for stream {}.", streamName);
+ LogReader reader = null;
+ Long lastTxId = null;
+ while (null == reader) {
+ // initialize the last txid
+ if (null == lastTxId) {
+ switch (readMode) {
+ case OLDEST:
+ lastTxId = 0L;
+ break;
+ case LATEST:
+ try {
+ lastTxId = dlm.getLastTxId();
+ } catch (IOException ioe) {
+ continue;
+ }
+ break;
+ case REWIND:
+ lastTxId = System.currentTimeMillis() - rewindMs;
+ break;
+ case POSITION:
+ lastTxId = fromTxId;
+ break;
+ default:
+ logger.warn("Unsupported mode {}", readMode);
+ printUsage();
+ System.exit(0);
+ break;
+ }
+ logger.info("Reading from transaction id {}", lastTxId);
+ }
+ // Open the reader
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ try {
+ reader = dlm.getInputStream(lastTxId);
+ long elapsedMs = stopwatch.elapsed(TimeUnit.MICROSECONDS);
+ openReaderStats.registerSuccessfulEvent(elapsedMs);
+ logger.info("It took {} ms to position the reader to transaction id {}", lastTxId);
+ } catch (IOException ioe) {
+ openReaderStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+ logger.warn("Failed to create reader for stream {} reading from {}.", streamName, lastTxId);
+ }
+ if (null == reader) {
+ try {
+ TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted from sleep after reader was reassigned null for stream {} : ",
+ streamName, e);
+ }
+ continue;
+ }
+
+ // read loop
+
+ LogRecord record;
+ boolean nonBlocking = false;
+ stopwatch = Stopwatch.createUnstarted();
+ long numCatchupReads = 0L;
+ long numCatchupBytes = 0L;
+ Stopwatch catchupStopwatch = Stopwatch.createStarted();
+ while (true) {
+ try {
+ stopwatch.start();
+ record = reader.readNext(nonBlocking);
+ if (null != record) {
+ long elapsedMicros = stopwatch.stop().elapsed(TimeUnit.MICROSECONDS);
+ if (nonBlocking) {
+ nonBlockingReadStats.registerSuccessfulEvent(elapsedMicros);
+ } else {
+ numCatchupBytes += record.getPayload().length;
+ ++numCatchupReads;
+ blockingReadStats.registerSuccessfulEvent(elapsedMicros);
+ }
+ lastTxId = record.getTransactionId();
+ } else {
+ nullReadCounter.inc();
+ }
+ if (null == record && !nonBlocking) {
+ nonBlocking = true;
+ catchupStopwatch.stop();
+ logger.info("Catchup {} records (total {} bytes) in {} milliseconds",
+ new Object[] { numCatchupReads, numCatchupBytes,
+ stopwatch.elapsed(TimeUnit.MILLISECONDS) });
+ }
+ stopwatch.reset();
+ } catch (IOException e) {
+ logger.warn("Encountered reading record from stream {} : ", streamName, e);
+ reader = null;
+ break;
+ }
+ }
+ try {
+ TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted from sleep while creating reader for stream {} : ",
+ streamName, e);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/package-info.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/package-info.java
new file mode 100644
index 0000000..b95a40f
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Stream level benchmarks.
+ */
+package org.apache.distributedlog.benchmark.stream;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/ShiftableRateLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/ShiftableRateLimiter.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/ShiftableRateLimiter.java
new file mode 100644
index 0000000..03c561c
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/ShiftableRateLimiter.java
@@ -0,0 +1,71 @@
+/**
+ * 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 org.apache.distributedlog.benchmark.utils;
+
+import com.google.common.util.concurrent.RateLimiter;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A wrapper over rate limiter.
+ */
+public class ShiftableRateLimiter implements Runnable {
+
+ private final RateLimiter rateLimiter;
+ private final ScheduledExecutorService executor;
+ private final double initialRate, maxRate, changeRate;
+ private final long changeInterval;
+ private final TimeUnit changeIntervalUnit;
+ private double nextRate;
+
+ public ShiftableRateLimiter(double initialRate,
+ double maxRate,
+ double changeRate,
+ long changeInterval,
+ TimeUnit changeIntervalUnit) {
+ this.initialRate = initialRate;
+ this.maxRate = maxRate;
+ this.changeRate = changeRate;
+ this.nextRate = initialRate;
+ this.changeInterval = changeInterval;
+ this.changeIntervalUnit = changeIntervalUnit;
+ this.rateLimiter = RateLimiter.create(initialRate);
+ this.executor = Executors.newSingleThreadScheduledExecutor();
+ this.executor.scheduleAtFixedRate(this, changeInterval, changeInterval, changeIntervalUnit);
+ }
+
+ public ShiftableRateLimiter duplicate() {
+ return new ShiftableRateLimiter(
+ initialRate,
+ maxRate,
+ changeRate,
+ changeInterval,
+ changeIntervalUnit);
+ }
+
+ @Override
+ public void run() {
+ this.nextRate = Math.min(nextRate + changeRate, maxRate);
+ this.rateLimiter.setRate(nextRate);
+ }
+
+ public RateLimiter getLimiter() {
+ return this.rateLimiter;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/package-info.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/package-info.java
new file mode 100644
index 0000000..c650bab
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Utils for benchmarking.
+ */
+package org.apache.distributedlog.benchmark.utils;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/resources/findbugsExclude.xml
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/resources/findbugsExclude.xml b/distributedlog-benchmark/src/main/resources/findbugsExclude.xml
index b7a1ecb..0ab2b6b 100644
--- a/distributedlog-benchmark/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-benchmark/src/main/resources/findbugsExclude.xml
@@ -18,6 +18,6 @@
<FindBugsFilter>
<Match>
<!-- generated code, we can't be held responsible for findbugs in it //-->
- <Class name="~com\.twitter\.distributedlog\.benchmark\.thrift.*" />
+ <Class name="~org\.apache\.distributedlog\.benchmark\.thrift.*" />
</Match>
</FindBugsFilter>
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/thrift/loadtest.thrift
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/thrift/loadtest.thrift b/distributedlog-benchmark/src/main/thrift/loadtest.thrift
index 6d98cec..48c5d5a 100644
--- a/distributedlog-benchmark/src/main/thrift/loadtest.thrift
+++ b/distributedlog-benchmark/src/main/thrift/loadtest.thrift
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-namespace java com.twitter.distributedlog.benchmark.thrift
+namespace java org.apache.distributedlog.benchmark.thrift
struct Message {
1: i64 publishTime;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-client/pom.xml b/distributedlog-client/pom.xml
index aad5093..f09caf1 100644
--- a/distributedlog-client/pom.xml
+++ b/distributedlog-client/pom.xml
@@ -137,7 +137,7 @@
<properties>
<property>
<name>listener</name>
- <value>com.twitter.distributedlog.TimedOutTestsListener</value>
+ <value>org.apache.distributedlog.TimedOutTestsListener</value>
</property>
</properties>
</configuration>
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ClientConfig.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ClientConfig.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ClientConfig.java
deleted file mode 100644
index de74f5a..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ClientConfig.java
+++ /dev/null
@@ -1,187 +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.client;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Client Config.
- */
-public class ClientConfig {
- int redirectBackoffStartMs = 25;
- int redirectBackoffMaxMs = 100;
- int maxRedirects = -1;
- int requestTimeoutMs = -1;
- boolean thriftmux = false;
- boolean streamFailfast = false;
- String streamNameRegex = ".*";
- boolean handshakeWithClientInfo = true;
- long periodicHandshakeIntervalMs = TimeUnit.MINUTES.toMillis(5);
- long periodicOwnershipSyncIntervalMs = TimeUnit.MINUTES.toMillis(5);
- boolean periodicDumpOwnershipCacheEnabled = false;
- long periodicDumpOwnershipCacheIntervalMs = TimeUnit.MINUTES.toMillis(10);
- boolean enableHandshakeTracing = false;
- boolean enableChecksum = true;
-
- public ClientConfig setMaxRedirects(int maxRedirects) {
- this.maxRedirects = maxRedirects;
- return this;
- }
-
- public int getMaxRedirects() {
- return this.maxRedirects;
- }
-
- public ClientConfig setRequestTimeoutMs(int timeoutInMillis) {
- this.requestTimeoutMs = timeoutInMillis;
- return this;
- }
-
- public int getRequestTimeoutMs() {
- return this.requestTimeoutMs;
- }
-
- public ClientConfig setRedirectBackoffStartMs(int ms) {
- this.redirectBackoffStartMs = ms;
- return this;
- }
-
- public int getRedirectBackoffStartMs() {
- return this.redirectBackoffStartMs;
- }
-
- public ClientConfig setRedirectBackoffMaxMs(int ms) {
- this.redirectBackoffMaxMs = ms;
- return this;
- }
-
- public int getRedirectBackoffMaxMs() {
- return this.redirectBackoffMaxMs;
- }
-
- public ClientConfig setThriftMux(boolean enabled) {
- this.thriftmux = enabled;
- return this;
- }
-
- public boolean getThriftMux() {
- return this.thriftmux;
- }
-
- public ClientConfig setStreamFailfast(boolean enabled) {
- this.streamFailfast = enabled;
- return this;
- }
-
- public boolean getStreamFailfast() {
- return this.streamFailfast;
- }
-
- public ClientConfig setStreamNameRegex(String nameRegex) {
- checkNotNull(nameRegex);
- this.streamNameRegex = nameRegex;
- return this;
- }
-
- public String getStreamNameRegex() {
- return this.streamNameRegex;
- }
-
- public ClientConfig setHandshakeWithClientInfo(boolean enabled) {
- this.handshakeWithClientInfo = enabled;
- return this;
- }
-
- public boolean getHandshakeWithClientInfo() {
- return this.handshakeWithClientInfo;
- }
-
- public ClientConfig setPeriodicHandshakeIntervalMs(long intervalMs) {
- this.periodicHandshakeIntervalMs = intervalMs;
- return this;
- }
-
- public long getPeriodicHandshakeIntervalMs() {
- return this.periodicHandshakeIntervalMs;
- }
-
- public ClientConfig setPeriodicOwnershipSyncIntervalMs(long intervalMs) {
- this.periodicOwnershipSyncIntervalMs = intervalMs;
- return this;
- }
-
- public long getPeriodicOwnershipSyncIntervalMs() {
- return this.periodicOwnershipSyncIntervalMs;
- }
-
- public ClientConfig setPeriodicDumpOwnershipCacheEnabled(boolean enabled) {
- this.periodicDumpOwnershipCacheEnabled = enabled;
- return this;
- }
-
- public boolean isPeriodicDumpOwnershipCacheEnabled() {
- return this.periodicDumpOwnershipCacheEnabled;
- }
-
- public ClientConfig setPeriodicDumpOwnershipCacheIntervalMs(long intervalMs) {
- this.periodicDumpOwnershipCacheIntervalMs = intervalMs;
- return this;
- }
-
- public long getPeriodicDumpOwnershipCacheIntervalMs() {
- return this.periodicDumpOwnershipCacheIntervalMs;
- }
-
- public ClientConfig setHandshakeTracingEnabled(boolean enabled) {
- this.enableHandshakeTracing = enabled;
- return this;
- }
-
- public boolean isHandshakeTracingEnabled() {
- return this.enableHandshakeTracing;
- }
-
- public ClientConfig setChecksumEnabled(boolean enabled) {
- this.enableChecksum = enabled;
- return this;
- }
-
- public boolean isChecksumEnabled() {
- return this.enableChecksum;
- }
-
- public static ClientConfig newConfig(ClientConfig config) {
- ClientConfig newConfig = new ClientConfig();
- newConfig.setMaxRedirects(config.getMaxRedirects())
- .setRequestTimeoutMs(config.getRequestTimeoutMs())
- .setRedirectBackoffStartMs(config.getRedirectBackoffStartMs())
- .setRedirectBackoffMaxMs(config.getRedirectBackoffMaxMs())
- .setThriftMux(config.getThriftMux())
- .setStreamFailfast(config.getStreamFailfast())
- .setStreamNameRegex(config.getStreamNameRegex())
- .setHandshakeWithClientInfo(config.getHandshakeWithClientInfo())
- .setPeriodicHandshakeIntervalMs(config.getPeriodicHandshakeIntervalMs())
- .setPeriodicDumpOwnershipCacheEnabled(config.isPeriodicDumpOwnershipCacheEnabled())
- .setPeriodicDumpOwnershipCacheIntervalMs(config.getPeriodicDumpOwnershipCacheIntervalMs())
- .setHandshakeTracingEnabled(config.isHandshakeTracingEnabled())
- .setChecksumEnabled(config.isChecksumEnabled());
- return newConfig;
- }
-}
[32/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
deleted file mode 100644
index 8d3c418..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
+++ /dev/null
@@ -1,289 +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.ZooKeeperClient.Credentials;
-import com.twitter.distributedlog.ZooKeeperClient.DigestCredentials;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.net.NetUtils;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy;
-import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.zookeeper.KeeperException;
-import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * BookKeeper Client wrapper over {@link BookKeeper}.
- *
- * <h3>Metrics</h3>
- * <ul>
- * <li> bookkeeper operation stats are exposed under current scope by {@link BookKeeper}
- * </ul>
- */
-public class BookKeeperClient {
- static final Logger LOG = LoggerFactory.getLogger(BookKeeperClient.class);
-
- // Parameters to build bookkeeper client
- private final DistributedLogConfiguration conf;
- private final String name;
- private final String zkServers;
- private final String ledgersPath;
- private final byte[] passwd;
- private final ClientSocketChannelFactory channelFactory;
- private final HashedWheelTimer requestTimer;
- private final StatsLogger statsLogger;
-
- // bookkeeper client state
- private boolean closed = false;
- private BookKeeper bkc = null;
- private ZooKeeperClient zkc;
- private final boolean ownZK;
- // feature provider
- private final Optional<FeatureProvider> featureProvider;
-
- @SuppressWarnings("deprecation")
- private synchronized void commonInitialization(
- DistributedLogConfiguration conf, String ledgersPath,
- ClientSocketChannelFactory channelFactory, StatsLogger statsLogger, HashedWheelTimer requestTimer)
- throws IOException, InterruptedException, KeeperException {
- ClientConfiguration bkConfig = new ClientConfiguration();
- bkConfig.setAddEntryTimeout(conf.getBKClientWriteTimeout());
- bkConfig.setReadTimeout(conf.getBKClientReadTimeout());
- bkConfig.setZkLedgersRootPath(ledgersPath);
- bkConfig.setZkTimeout(conf.getBKClientZKSessionTimeoutMilliSeconds());
- bkConfig.setNumWorkerThreads(conf.getBKClientNumberWorkerThreads());
- bkConfig.setEnsemblePlacementPolicy(RegionAwareEnsemblePlacementPolicy.class);
- bkConfig.setZkRequestRateLimit(conf.getBKClientZKRequestRateLimit());
- bkConfig.setProperty(RegionAwareEnsemblePlacementPolicy.REPP_DISALLOW_BOOKIE_PLACEMENT_IN_REGION_FEATURE_NAME,
- DistributedLogConstants.DISALLOW_PLACEMENT_IN_REGION_FEATURE_NAME);
- // reload configuration from dl configuration with settings prefixed with 'bkc.'
- ConfUtils.loadConfiguration(bkConfig, conf, "bkc.");
-
- Class<? extends DNSToSwitchMapping> dnsResolverCls;
- try {
- dnsResolverCls = conf.getEnsemblePlacementDnsResolverClass();
- } catch (ConfigurationException e) {
- LOG.error("Failed to load bk dns resolver : ", e);
- throw new IOException("Failed to load bk dns resolver : ", e);
- }
- final DNSToSwitchMapping dnsResolver =
- NetUtils.getDNSResolver(dnsResolverCls, conf.getBkDNSResolverOverrides());
-
- this.bkc = BookKeeper.newBuilder()
- .config(bkConfig)
- .zk(zkc.get())
- .channelFactory(channelFactory)
- .statsLogger(statsLogger)
- .dnsResolver(dnsResolver)
- .requestTimer(requestTimer)
- .featureProvider(featureProvider.orNull())
- .build();
- }
-
- BookKeeperClient(DistributedLogConfiguration conf,
- String name,
- String zkServers,
- ZooKeeperClient zkc,
- String ledgersPath,
- ClientSocketChannelFactory channelFactory,
- HashedWheelTimer requestTimer,
- StatsLogger statsLogger,
- Optional<FeatureProvider> featureProvider) {
- this.conf = conf;
- this.name = name;
- this.zkServers = zkServers;
- this.ledgersPath = ledgersPath;
- this.passwd = conf.getBKDigestPW().getBytes(UTF_8);
- this.channelFactory = channelFactory;
- this.requestTimer = requestTimer;
- this.statsLogger = statsLogger;
- this.featureProvider = featureProvider;
- this.ownZK = null == zkc;
- if (null != zkc) {
- // reference the passing zookeeper client
- this.zkc = zkc;
- }
- }
-
- private synchronized void initialize() throws IOException {
- if (null != this.bkc) {
- return;
- }
- if (null == this.zkc) {
- int zkSessionTimeout = conf.getBKClientZKSessionTimeoutMilliSeconds();
- RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
- conf.getBKClientZKRetryBackoffStartMillis(),
- conf.getBKClientZKRetryBackoffMaxMillis(), conf.getBKClientZKNumRetries());
- Credentials credentials = Credentials.NONE;
- if (conf.getZkAclId() != null) {
- credentials = new DigestCredentials(conf.getZkAclId(), conf.getZkAclId());
- }
-
- this.zkc = new ZooKeeperClient(name + ":zk", zkSessionTimeout, 2 * zkSessionTimeout, zkServers,
- retryPolicy, statsLogger.scope("bkc_zkc"), conf.getZKClientNumberRetryThreads(),
- conf.getBKClientZKRequestRateLimit(), credentials);
- }
-
- try {
- commonInitialization(conf, ledgersPath, channelFactory, statsLogger, requestTimer);
- } catch (InterruptedException e) {
- throw new DLInterruptedException("Interrupted on creating bookkeeper client " + name + " : ", e);
- } catch (KeeperException e) {
- throw new ZKException("Error on creating bookkeeper client " + name + " : ", e);
- }
-
- if (ownZK) {
- LOG.info("BookKeeper Client created {} with its own ZK Client : ledgersPath = {}, numRetries = {}, " +
- "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
- new Object[] { name, ledgersPath,
- conf.getBKClientZKNumRetries(), conf.getBKClientZKSessionTimeoutMilliSeconds(),
- conf.getBKClientZKRetryBackoffStartMillis(), conf.getBKClientZKRetryBackoffMaxMillis(),
- conf.getBkDNSResolverOverrides() });
- } else {
- LOG.info("BookKeeper Client created {} with shared zookeeper client : ledgersPath = {}, numRetries = {}, " +
- "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
- new Object[] { name, ledgersPath,
- conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(),
- conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(),
- conf.getBkDNSResolverOverrides() });
- }
- }
-
-
- public synchronized BookKeeper get() throws IOException {
- checkClosedOrInError();
- if (null == bkc) {
- initialize();
- }
- return bkc;
- }
-
- // Util functions
- public Future<LedgerHandle> createLedger(int ensembleSize,
- int writeQuorumSize,
- int ackQuorumSize) {
- BookKeeper bk;
- try {
- bk = get();
- } catch (IOException ioe) {
- return Future.exception(ioe);
- }
- final Promise<LedgerHandle> promise = new Promise<LedgerHandle>();
- bk.asyncCreateLedger(ensembleSize, writeQuorumSize, ackQuorumSize,
- BookKeeper.DigestType.CRC32, passwd, new AsyncCallback.CreateCallback() {
- @Override
- public void createComplete(int rc, LedgerHandle lh, Object ctx) {
- if (BKException.Code.OK == rc) {
- promise.updateIfEmpty(new Return<LedgerHandle>(lh));
- } else {
- promise.updateIfEmpty(new Throw<LedgerHandle>(BKException.create(rc)));
- }
- }
- }, null);
- return promise;
- }
-
- public Future<Void> deleteLedger(long lid,
- final boolean ignoreNonExistentLedger) {
- BookKeeper bk;
- try {
- bk = get();
- } catch (IOException ioe) {
- return Future.exception(ioe);
- }
- final Promise<Void> promise = new Promise<Void>();
- bk.asyncDeleteLedger(lid, new AsyncCallback.DeleteCallback() {
- @Override
- public void deleteComplete(int rc, Object ctx) {
- if (BKException.Code.OK == rc) {
- promise.updateIfEmpty(new Return<Void>(null));
- } else if (BKException.Code.NoSuchLedgerExistsException == rc) {
- if (ignoreNonExistentLedger) {
- promise.updateIfEmpty(new Return<Void>(null));
- } else {
- promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
- }
- } else {
- promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
- }
- }
- }, null);
- return promise;
- }
-
- 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 != bkcToClose) {
- try {
- bkcToClose.close();
- } catch (InterruptedException e) {
- LOG.warn("Interrupted on closing bookkeeper client {} : ", name, e);
- Thread.currentThread().interrupt();
- } catch (BKException e) {
- LOG.warn("Error on closing bookkeeper client {} : ", name, e);
- }
- }
- if (null != zkcToClose) {
- if (ownZK) {
- zkcToClose.close();
- }
- }
- }
-
- public synchronized void checkClosedOrInError() throws AlreadyClosedException {
- if (closed) {
- LOG.error("BookKeeper Client {} is already closed", name);
- throw new AlreadyClosedException("BookKeeper Client " + name + " is already closed");
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClientBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClientBuilder.java
deleted file mode 100644
index cad1096..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClientBuilder.java
+++ /dev/null
@@ -1,209 +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.google.common.base.Preconditions;
-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.apache.bookkeeper.feature.FeatureProvider;
-
-import org.apache.bookkeeper.feature.Feature;
-
-/**
- * Builder to build bookkeeper client.
- */
-public class BookKeeperClientBuilder {
-
- /**
- * Create a bookkeeper client builder to build bookkeeper clients.
- *
- * @return bookkeeper client builder.
- */
- public static BookKeeperClientBuilder newBuilder() {
- return new BookKeeperClientBuilder();
- }
-
- // client name
- private String name = null;
- // dl config
- private DistributedLogConfiguration dlConfig = null;
- // bookkeeper settings
- // zookeeper client
- private ZooKeeperClient zkc = null;
- // or zookeeper servers
- private String zkServers = null;
- // ledgers path
- private String ledgersPath = null;
- // statsLogger
- private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
- // client channel factory
- private ClientSocketChannelFactory channelFactory = null;
- // request timer
- private HashedWheelTimer requestTimer = null;
- // feature provider
- private Optional<FeatureProvider> featureProvider = Optional.absent();
-
- // Cached BookKeeper Client
- private BookKeeperClient cachedClient = null;
-
- /**
- * Private bookkeeper builder.
- */
- private BookKeeperClientBuilder() {}
-
- /**
- * Set client name.
- *
- * @param name
- * client name.
- * @return builder
- */
- public synchronized BookKeeperClientBuilder name(String name) {
- this.name = name;
- return this;
- }
-
- /**
- * <i>dlConfig</i> used to configure bookkeeper client.
- *
- * @param dlConfig
- * distributedlog config.
- * @return builder.
- */
- public synchronized BookKeeperClientBuilder dlConfig(DistributedLogConfiguration dlConfig) {
- this.dlConfig = dlConfig;
- return this;
- }
-
- /**
- * Set the zkc used to build bookkeeper client. If a zookeeper client is provided in this
- * method, bookkeeper client will use it rather than creating a brand new one.
- *
- * @param zkc
- * zookeeper client.
- * @return builder
- * @see #zkServers(String)
- */
- public synchronized BookKeeperClientBuilder zkc(ZooKeeperClient zkc) {
- this.zkc = zkc;
- return this;
- }
-
- /**
- * Set the zookeeper servers that bookkeeper client would connect to. If no zookeeper client
- * is provided by {@link #zkc(ZooKeeperClient)}, bookkeeper client will use the given string
- * to create a brand new zookeeper client.
- *
- * @param zkServers
- * zookeeper servers that bookkeeper client would connect to.
- * @return builder
- * @see #zkc(ZooKeeperClient)
- */
- public synchronized BookKeeperClientBuilder zkServers(String zkServers) {
- this.zkServers = zkServers;
- return this;
- }
-
- /**
- * Set the ledgers path that bookkeeper client is going to access.
- *
- * @param ledgersPath
- * ledgers path
- * @return builder
- * @see org.apache.bookkeeper.conf.ClientConfiguration#getZkLedgersRootPath()
- */
- public synchronized BookKeeperClientBuilder ledgersPath(String ledgersPath) {
- this.ledgersPath = ledgersPath;
- return this;
- }
-
- /**
- * Build BookKeeper client using existing <i>bkc</i> client.
- *
- * @param bkc
- * bookkeeper client.
- * @return builder
- */
- public synchronized BookKeeperClientBuilder bkc(BookKeeperClient bkc) {
- this.cachedClient = bkc;
- return this;
- }
-
- /**
- * Build BookKeeper client using existing <i>channelFactory</i>.
- *
- * @param channelFactory
- * Channel Factory used to build bookkeeper client.
- * @return bookkeeper client builder.
- */
- public synchronized BookKeeperClientBuilder channelFactory(ClientSocketChannelFactory channelFactory) {
- this.channelFactory = channelFactory;
- return this;
- }
-
- /**
- * Build BookKeeper client using existing <i>request timer</i>.
- *
- * @param requestTimer
- * HashedWheelTimer used to build bookkeeper client.
- * @return bookkeeper client builder.
- */
- public synchronized BookKeeperClientBuilder requestTimer(HashedWheelTimer requestTimer) {
- this.requestTimer = requestTimer;
- return this;
- }
-
- /**
- * Build BookKeeper Client using given stats logger <i>statsLogger</i>.
- *
- * @param statsLogger
- * stats logger to report stats
- * @return builder.
- */
- public synchronized BookKeeperClientBuilder statsLogger(StatsLogger statsLogger) {
- this.statsLogger = statsLogger;
- return this;
- }
-
- public synchronized BookKeeperClientBuilder featureProvider(Optional<FeatureProvider> featureProvider) {
- this.featureProvider = featureProvider;
- return this;
- }
-
- private void validateParameters() {
- Preconditions.checkNotNull(name, "Missing client name.");
- Preconditions.checkNotNull(dlConfig, "Missing DistributedLog Configuration.");
- Preconditions.checkArgument(null == zkc || null == zkServers, "Missing zookeeper setting.");
- Preconditions.checkNotNull(ledgersPath, "Missing Ledgers Root Path.");
- }
-
- public synchronized BookKeeperClient build() {
- if (null == cachedClient) {
- cachedClient = buildClient();
- }
- return cachedClient;
- }
-
- private BookKeeperClient buildClient() {
- validateParameters();
- return new BookKeeperClient(dlConfig, name, zkServers, zkc, ledgersPath, channelFactory, requestTimer, statsLogger, featureProvider);
- }
-}
[16/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 03d70bd..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
+++ /dev/null
@@ -1,2873 +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.tools;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-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;
-import org.apache.bookkeeper.client.BookKeeperAccessor;
-import org.apache.bookkeeper.client.BookKeeperAdmin;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
-import org.apache.bookkeeper.client.LedgerReader;
-import org.apache.bookkeeper.net.BookieSocketAddress;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.codec.binary.Hex;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.AsyncLogReader;
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.BookKeeperClientBuilder;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.LogReader;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.auditor.DLAuditor;
-import com.twitter.distributedlog.bk.LedgerAllocator;
-import com.twitter.distributedlog.bk.LedgerAllocatorUtils;
-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.FutureEventListener;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-public class DistributedLogTool extends Tool {
-
- static final Logger logger = LoggerFactory.getLogger(DistributedLogTool.class);
-
- static final List<String> EMPTY_LIST = Lists.newArrayList();
-
- static int compareByCompletionTime(long time1, long time2) {
- return time1 > time2 ? 1 : (time1 < time2 ? -1 : 0);
- }
-
- static final Comparator<LogSegmentMetadata> LOGSEGMENT_COMPARATOR_BY_TIME = new Comparator<LogSegmentMetadata>() {
- @Override
- public int compare(LogSegmentMetadata o1, LogSegmentMetadata o2) {
- if (o1.isInProgress() && o2.isInProgress()) {
- return compareByCompletionTime(o1.getFirstTxId(), o2.getFirstTxId());
- } else if (!o1.isInProgress() && !o2.isInProgress()) {
- return compareByCompletionTime(o1.getCompletionTime(), o2.getCompletionTime());
- } else if (o1.isInProgress() && !o2.isInProgress()) {
- return compareByCompletionTime(o1.getFirstTxId(), o2.getCompletionTime());
- } else {
- return compareByCompletionTime(o1.getCompletionTime(), o2.getFirstTxId());
- }
- }
- };
-
- static DLSN parseDLSN(String dlsnStr) throws ParseException {
- if (dlsnStr.equals("InitialDLSN")) {
- return DLSN.InitialDLSN;
- }
- String[] parts = dlsnStr.split(",");
- if (parts.length != 3) {
- throw new ParseException("Invalid dlsn : " + dlsnStr);
- }
- try {
- return new DLSN(Long.parseLong(parts[0]), Long.parseLong(parts[1]), Long.parseLong(parts[2]));
- } catch (Exception nfe) {
- throw new ParseException("Invalid dlsn : " + dlsnStr);
- }
- }
-
- /**
- * Per DL Command, which parses basic options. e.g. uri.
- */
- protected abstract static class PerDLCommand extends OptsCommand {
-
- protected Options options = new Options();
- protected final DistributedLogConfiguration dlConf;
- protected URI uri;
- protected String zkAclId = null;
- protected boolean force = false;
- protected DistributedLogNamespace namespace = null;
-
- protected PerDLCommand(String name, String description) {
- super(name, description);
- dlConf = new DistributedLogConfiguration();
- // Tools are allowed to read old metadata as long as they can interpret it
- dlConf.setDLLedgerMetadataSkipMinVersionCheck(true);
- options.addOption("u", "uri", true, "DistributedLog URI");
- options.addOption("c", "conf", true, "DistributedLog Configuration File");
- options.addOption("a", "zk-acl-id", true, "Zookeeper ACL ID");
- options.addOption("f", "force", false, "Force command (no warnings or prompts)");
- }
-
- @Override
- protected int runCmd(CommandLine commandLine) throws Exception {
- try {
- parseCommandLine(commandLine);
- } catch (ParseException pe) {
- System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
- printUsage();
- return -1;
- }
- try {
- return runCmd();
- } finally {
- if (null != namespace) {
- namespace.close();
- }
- }
- }
-
- protected abstract int runCmd() throws Exception;
-
- @Override
- protected Options getOptions() {
- return options;
- }
-
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- if (!cmdline.hasOption("u")) {
- throw new ParseException("No distributedlog uri provided.");
- }
- uri = URI.create(cmdline.getOptionValue("u"));
- if (cmdline.hasOption("c")) {
- String configFile = cmdline.getOptionValue("c");
- try {
- dlConf.loadConf(new File(configFile).toURI().toURL());
- } catch (ConfigurationException e) {
- throw new ParseException("Failed to load distributedlog configuration from " + configFile + ".");
- } catch (MalformedURLException e) {
- throw new ParseException("Failed to load distributedlog configuration from " + configFile + ": malformed uri.");
- }
- }
- if (cmdline.hasOption("a")) {
- zkAclId = cmdline.getOptionValue("a");
- }
- if (cmdline.hasOption("f")) {
- force = true;
- }
- }
-
- protected DistributedLogConfiguration getConf() {
- return dlConf;
- }
-
- protected URI getUri() {
- return uri;
- }
-
- protected void setUri(URI uri) {
- this.uri = uri;
- }
-
- protected String getZkAclId() {
- return zkAclId;
- }
-
- protected void setZkAclId(String zkAclId) {
- this.zkAclId = zkAclId;
- }
-
- protected boolean getForce() {
- return force;
- }
-
- protected void setForce(boolean force) {
- this.force = force;
- }
-
- protected DistributedLogNamespace getNamespace() throws IOException {
- if (null == this.namespace) {
- this.namespace = DistributedLogNamespaceBuilder.newBuilder()
- .uri(getUri())
- .conf(getConf())
- .build();
- }
- return this.namespace;
- }
-
- protected LogSegmentMetadataStore getLogSegmentMetadataStore() throws IOException {
- return getNamespace()
- .getNamespaceDriver()
- .getLogStreamMetadataStore(NamespaceDriver.Role.READER)
- .getLogSegmentMetadataStore();
- }
-
- protected ZooKeeperClient getZooKeeperClient() throws IOException {
- NamespaceDriver driver = getNamespace().getNamespaceDriver();
- assert(driver instanceof BKNamespaceDriver);
- return ((BKNamespaceDriver) driver).getWriterZKC();
- }
-
- protected BookKeeperClient getBookKeeperClient() throws IOException {
- NamespaceDriver driver = getNamespace().getNamespaceDriver();
- assert(driver instanceof BKNamespaceDriver);
- return ((BKNamespaceDriver) driver).getReaderBKC();
- }
- }
-
- /**
- * Base class for simple command with no resource setup requirements.
- */
- public abstract static class SimpleCommand extends OptsCommand {
-
- protected final Options options = new Options();
-
- SimpleCommand(String name, String description) {
- super(name, description);
- }
-
- @Override
- protected int runCmd(CommandLine commandLine) throws Exception {
- try {
- parseCommandLine(commandLine);
- } catch (ParseException pe) {
- System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
- printUsage();
- return -1;
- }
- return runSimpleCmd();
- }
-
- abstract protected int runSimpleCmd() throws Exception;
-
- abstract protected void parseCommandLine(CommandLine cmdline) throws ParseException;
-
- @Override
- protected Options getOptions() {
- return options;
- }
- }
-
- /**
- * Per Stream Command, which parse common options for per stream. e.g. stream name.
- */
- abstract static class PerStreamCommand extends PerDLCommand {
-
- protected String streamName;
-
- protected PerStreamCommand(String name, String description) {
- super(name, description);
- options.addOption("s", "stream", true, "Stream Name");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- if (!cmdline.hasOption("s")) {
- throw new ParseException("No stream name provided.");
- }
- streamName = cmdline.getOptionValue("s");
- }
-
- protected String getStreamName() {
- return streamName;
- }
-
- protected void setStreamName(String streamName) {
- this.streamName = streamName;
- }
- }
-
- /**
- * 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;
- String allocationPoolPath = DistributedLogConstants.ALLOCATION_POOL_NODE;
-
- DeleteAllocatorPoolCommand() {
- super("delete_allocator_pool", "Delete allocator pool for a given distributedlog instance");
- options.addOption("t", "concurrency", true, "Concurrency on deleting allocator pool.");
- options.addOption("ap", "allocation-pool-path", true, "Ledger Allocation Pool Path");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- if (cmdline.hasOption("t")) {
- concurrency = Integer.parseInt(cmdline.getOptionValue("t"));
- if (concurrency <= 0) {
- throw new ParseException("Invalid concurrency value : " + concurrency + ": it must be greater or equal to 0.");
- }
- }
- if (cmdline.hasOption("ap")) {
- allocationPoolPath = cmdline.getOptionValue("ap");
- if (!allocationPoolPath.startsWith(".") || !allocationPoolPath.contains("allocation")) {
- throw new ParseException("Invalid allocation pool path : " + allocationPoolPath + ": it must starts with a '.' and must contains 'allocation'");
- }
- }
- }
-
- @Override
- protected int runCmd() throws Exception {
- 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 = 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) {
- poolsToDelete.add(rootPath + "/" + pool);
- }
- final CountDownLatch doneLatch = new CountDownLatch(concurrency);
- for (int i = 0; i < concurrency; i++) {
- final int tid = i;
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- while (!poolsToDelete.isEmpty()) {
- String poolPath = poolsToDelete.poll();
- if (null == poolPath) {
- break;
- }
- try {
- LedgerAllocator allocator =
- LedgerAllocatorUtils.createLedgerAllocatorPool(poolPath, 0, getConf(),
- zkc, bkc,
- allocationExecutor);
- allocator.delete();
- System.out.println("Deleted allocator pool : " + poolPath + " .");
- } catch (IOException ioe) {
- System.err.println("Failed to delete allocator pool " + poolPath + " : " + ioe.getMessage());
- }
- }
- doneLatch.countDown();
- System.out.println("Thread " + tid + " is done.");
- }
- });
- }
- doneLatch.await();
- }
- } finally {
- executorService.shutdown();
- allocationExecutor.shutdown();
- }
- return 0;
- }
-
- @Override
- protected String getUsage() {
- return "delete_allocator_pool";
- }
- }
-
- public static class ListCommand extends PerDLCommand {
-
- boolean printMetadata = false;
- boolean printHex = false;
-
- ListCommand() {
- super("list", "list streams of a given distributedlog instance");
- options.addOption("m", "meta", false, "Print metadata associated with each stream");
- options.addOption("x", "hex", false, "Print metadata in hex format");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- printMetadata = cmdline.hasOption("m");
- printHex = cmdline.hasOption("x");
- }
-
- @Override
- protected String getUsage() {
- return "list [options]";
- }
-
- @Override
- protected int runCmd() throws Exception {
- printStreams(getNamespace());
- return 0;
- }
-
- protected void printStreams(DistributedLogNamespace namespace) throws Exception {
- Iterator<String> streams = namespace.getLogs();
- System.out.println("Streams under " + getUri() + " : ");
- System.out.println("--------------------------------");
- 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(metadata));
- } else {
- System.out.println(new String(metadata, UTF_8));
- }
- System.out.println("");
- }
- System.out.println("--------------------------------");
- }
- }
-
- public static class WatchNamespaceCommand extends PerDLCommand implements NamespaceListener {
- private Set<String> currentSet = Sets.<String>newHashSet();
- private CountDownLatch doneLatch = new CountDownLatch(1);
-
- WatchNamespaceCommand() {
- super("watch", "watch and report changes for a dl namespace");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- }
-
- @Override
- protected String getUsage() {
- return "watch [options]";
- }
-
- @Override
- protected int runCmd() throws Exception {
- watchAndReportChanges(getNamespace());
- doneLatch.await();
- return 0;
- }
-
- @Override
- public synchronized void onStreamsChanged(Iterator<String> streams) {
- Set<String> updatedSet = Sets.newHashSet(streams);
- Set<String> oldStreams = Sets.difference(currentSet, updatedSet);
- Set<String> newStreams = Sets.difference(updatedSet, currentSet);
- currentSet = updatedSet;
-
- System.out.println("Old streams : ");
- for (String stream : oldStreams) {
- System.out.println(stream);
- }
-
- System.out.println("New streams : ");
- for (String stream : newStreams) {
- System.out.println(stream);
- }
-
- System.out.println("");
- }
-
- protected void watchAndReportChanges(DistributedLogNamespace namespace) throws Exception {
- namespace.registerNamespaceListener(this);
- }
- }
-
- protected static class InspectCommand extends PerDLCommand {
-
- int numThreads = 1;
- String streamPrefix = null;
- boolean printInprogressOnly = false;
- boolean dumpEntries = false;
- boolean orderByTime = false;
- boolean printStreamsOnly = false;
- boolean checkInprogressOnly = false;
-
- InspectCommand() {
- super("inspect", "Inspect streams under a given dl uri to find any potential corruptions");
- options.addOption("t", "threads", true, "Number threads to do inspection.");
- options.addOption("ft", "filter", true, "Stream filter by prefix");
- options.addOption("i", "inprogress", false, "Print inprogress log segments only");
- options.addOption("d", "dump", false, "Dump entries of inprogress log segments");
- options.addOption("ot", "orderbytime", false, "Order the log segments by completion time");
- options.addOption("pso", "print-stream-only", false, "Print streams only");
- options.addOption("cio", "check-inprogress-only", false, "Check duplicated inprogress only");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- if (cmdline.hasOption("t")) {
- numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
- }
- if (cmdline.hasOption("ft")) {
- streamPrefix = cmdline.getOptionValue("ft");
- }
- printInprogressOnly = cmdline.hasOption("i");
- dumpEntries = cmdline.hasOption("d");
- orderByTime = cmdline.hasOption("ot");
- printStreamsOnly = cmdline.hasOption("pso");
- checkInprogressOnly = cmdline.hasOption("cio");
- }
-
- @Override
- protected int runCmd() throws Exception {
- SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates =
- new TreeMap<String, List<Pair<LogSegmentMetadata, List<String>>>>();
- inspectStreams(corruptedCandidates);
- System.out.println("Corrupted Candidates : ");
- if (printStreamsOnly) {
- System.out.println(corruptedCandidates.keySet());
- return 0;
- }
- for (Map.Entry<String, List<Pair<LogSegmentMetadata, List<String>>>> entry : corruptedCandidates.entrySet()) {
- System.out.println(entry.getKey() + " : \n");
- for (Pair<LogSegmentMetadata, List<String>> pair : entry.getValue()) {
- System.out.println("\t - " + pair.getLeft());
- if (printInprogressOnly && dumpEntries) {
- int i = 0;
- for (String entryData : pair.getRight()) {
- System.out.println("\t" + i + "\t: " + entryData);
- ++i;
- }
- }
- }
- System.out.println();
- }
- return 0;
- }
-
- private void inspectStreams(final SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates)
- throws Exception {
- Iterator<String> streamCollection = getNamespace().getLogs();
- final List<String> streams = new ArrayList<String>();
- while (streamCollection.hasNext()) {
- String s = streamCollection.next();
- if (null != streamPrefix) {
- if (s.startsWith(streamPrefix)) {
- streams.add(s);
- }
- } else {
- streams.add(s);
- }
- }
- if (0 == streams.size()) {
- return;
- }
- println("Streams : " + streams);
- if (!getForce() && !IOUtils.confirmPrompt("Are you sure you want to inspect " + streams.size() + " streams")) {
- return;
- }
- numThreads = Math.min(streams.size(), numThreads);
- final int numStreamsPerThreads = streams.size() / numThreads;
- Thread[] threads = new Thread[numThreads];
- for (int i = 0; i < numThreads; i++) {
- final int tid = i;
- threads[i] = new Thread("Inspect-" + i) {
- @Override
- public void run() {
- try {
- inspectStreams(streams, tid, numStreamsPerThreads, corruptedCandidates);
- System.out.println("Thread " + tid + " finished.");
- } catch (Exception e) {
- System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
- }
- }
- };
- threads[i].start();
- }
- for (int i = 0; i < numThreads; i++) {
- threads[i].join();
- }
- }
-
- private void inspectStreams(List<String> streams,
- int tid,
- int numStreamsPerThreads,
- SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates)
- throws Exception {
- 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);
- BookKeeperClient bkc = getBookKeeperClient();
- DistributedLogManager dlm = getNamespace().openLog(s);
- try {
- List<LogSegmentMetadata> segments = dlm.getLogSegments();
- if (segments.size() <= 1) {
- continue;
- }
- boolean isCandidate = false;
- if (checkInprogressOnly) {
- Set<Long> inprogressSeqNos = new HashSet<Long>();
- for (LogSegmentMetadata segment : segments) {
- if (segment.isInProgress()) {
- inprogressSeqNos.add(segment.getLogSegmentSequenceNumber());
- }
- }
- for (LogSegmentMetadata segment : segments) {
- if (!segment.isInProgress() && inprogressSeqNos.contains(segment.getLogSegmentSequenceNumber())) {
- isCandidate = true;
- }
- }
- } else {
- LogSegmentMetadata firstSegment = segments.get(0);
- long lastSeqNo = firstSegment.getLogSegmentSequenceNumber();
-
- for (int j = 1; j < segments.size(); j++) {
- LogSegmentMetadata nextSegment = segments.get(j);
- if (lastSeqNo + 1 != nextSegment.getLogSegmentSequenceNumber()) {
- isCandidate = true;
- break;
- }
- ++lastSeqNo;
- }
- }
- if (isCandidate) {
- if (orderByTime) {
- Collections.sort(segments, LOGSEGMENT_COMPARATOR_BY_TIME);
- }
- List<Pair<LogSegmentMetadata, List<String>>> ledgers =
- new ArrayList<Pair<LogSegmentMetadata, List<String>>>();
- for (LogSegmentMetadata seg : segments) {
- LogSegmentMetadata segment = seg;
- List<String> dumpedEntries = new ArrayList<String>();
- if (segment.isInProgress()) {
- LedgerHandle lh = bkc.get().openLedgerNoRecovery(segment.getLogSegmentId(), BookKeeper.DigestType.CRC32,
- dlConf.getBKDigestPW().getBytes(UTF_8));
- try {
- long lac = lh.readLastConfirmed();
- segment = segment.mutator().setLastEntryId(lac).build();
- if (printInprogressOnly && dumpEntries && lac >= 0) {
- Enumeration<LedgerEntry> entries = lh.readEntries(0L, lac);
- while (entries.hasMoreElements()) {
- LedgerEntry entry = entries.nextElement();
- dumpedEntries.add(new String(entry.getEntry(), UTF_8));
- }
- }
- } finally {
- lh.close();
- }
- }
- if (printInprogressOnly) {
- if (segment.isInProgress()) {
- ledgers.add(Pair.of(segment, dumpedEntries));
- }
- } else {
- ledgers.add(Pair.of(segment, EMPTY_LIST));
- }
- }
- synchronized (corruptedCandidates) {
- corruptedCandidates.put(s, ledgers);
- }
- }
- } finally {
- dlm.close();
- }
- }
- }
-
- @Override
- protected String getUsage() {
- return "inspect [options]";
- }
- }
-
- protected static class TruncateCommand extends PerDLCommand {
-
- int numThreads = 1;
- String streamPrefix = null;
- boolean deleteStream = false;
-
- TruncateCommand() {
- super("truncate", "truncate streams under a given dl uri");
- options.addOption("t", "threads", true, "Number threads to do truncation");
- options.addOption("ft", "filter", true, "Stream filter by prefix");
- options.addOption("d", "delete", false, "Delete Stream");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- if (cmdline.hasOption("t")) {
- numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
- }
- if (cmdline.hasOption("ft")) {
- streamPrefix = cmdline.getOptionValue("ft");
- }
- if (cmdline.hasOption("d")) {
- deleteStream = true;
- }
- }
-
- @Override
- protected String getUsage() {
- return "truncate [options]";
- }
-
- protected void setFilter(String filter) {
- this.streamPrefix = filter;
- }
-
- @Override
- protected int runCmd() throws Exception {
- getConf().setZkAclId(getZkAclId());
- return truncateStreams(getNamespace());
- }
-
- private int truncateStreams(final DistributedLogNamespace namespace) throws Exception {
- Iterator<String> streamCollection = namespace.getLogs();
- final List<String> streams = new ArrayList<String>();
- while (streamCollection.hasNext()) {
- String s = streamCollection.next();
- if (null != streamPrefix) {
- if (s.startsWith(streamPrefix)) {
- streams.add(s);
- }
- } else {
- streams.add(s);
- }
- }
- if (0 == streams.size()) {
- return 0;
- }
- System.out.println("Streams : " + streams);
- if (!getForce() && !IOUtils.confirmPrompt("Do you want to truncate " + streams.size() + " streams ?")) {
- return 0;
- }
- numThreads = Math.min(streams.size(), numThreads);
- final int numStreamsPerThreads = streams.size() / numThreads + 1;
- Thread[] threads = new Thread[numThreads];
- for (int i = 0; i < numThreads; i++) {
- final int tid = i;
- threads[i] = new Thread("Truncate-" + i) {
- @Override
- public void run() {
- try {
- truncateStreams(namespace, streams, tid, numStreamsPerThreads);
- System.out.println("Thread " + tid + " finished.");
- } catch (IOException e) {
- System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
- }
- }
- };
- threads[i].start();
- }
- for (int i = 0; i < numThreads; i++) {
- threads[i].join();
- }
- return 0;
- }
-
- 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 = namespace.openLog(s);
- try {
- if (deleteStream) {
- dlm.delete();
- } else {
- dlm.purgeLogsOlderThan(Long.MAX_VALUE);
- }
- } finally {
- dlm.close();
- }
- }
- }
- }
-
- public static class SimpleBookKeeperClient {
- BookKeeperClient bkc;
- ZooKeeperClient zkc;
-
- public SimpleBookKeeperClient(DistributedLogConfiguration conf, URI uri) {
- try {
- zkc = ZooKeeperClientBuilder.newBuilder()
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .zkAclId(conf.getZkAclId())
- .uri(uri)
- .build();
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
- BKDLConfig.propagateConfiguration(bkdlConfig, conf);
- bkc = BookKeeperClientBuilder.newBuilder()
- .zkc(zkc)
- .dlConfig(conf)
- .ledgersPath(bkdlConfig.getBkLedgersPath())
- .name("dlog")
- .build();
- } catch (Exception e) {
- close();
- }
- }
- public BookKeeperClient client() {
- return bkc;
- }
- public void close() {
- if (null != bkc) {
- bkc.close();
- }
- if (null != zkc) {
- zkc.close();
- }
- }
- }
-
- protected static class ShowCommand extends PerStreamCommand {
-
- SimpleBookKeeperClient bkc = null;
- boolean listSegments = true;
- boolean listEppStats = false;
- long firstLid = 0;
- long lastLid = -1;
-
- ShowCommand() {
- super("show", "show metadata of a given stream and list segments");
- options.addOption("ns", "no-log-segments", false, "Do not list log segment metadata");
- options.addOption("lp", "placement-stats", false, "Show ensemble placement stats");
- options.addOption("fl", "first-ledger", true, "First log sement no");
- options.addOption("ll", "last-ledger", true, "Last log sement no");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- if (cmdline.hasOption("fl")) {
- try {
- firstLid = Long.parseLong(cmdline.getOptionValue("fl"));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid ledger id " + cmdline.getOptionValue("fl"));
- }
- }
- if (firstLid < 0) {
- throw new IllegalArgumentException("Invalid ledger id " + firstLid);
- }
- if (cmdline.hasOption("ll")) {
- try {
- lastLid = Long.parseLong(cmdline.getOptionValue("ll"));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid ledger id " + cmdline.getOptionValue("ll"));
- }
- }
- if (lastLid != -1 && firstLid > lastLid) {
- throw new IllegalArgumentException("Invalid ledger ids " + firstLid + " " + lastLid);
- }
- listSegments = !cmdline.hasOption("ns");
- listEppStats = cmdline.hasOption("lp");
- }
-
- @Override
- protected int runCmd() throws Exception {
- DistributedLogManager dlm = getNamespace().openLog(getStreamName());
- try {
- if (listEppStats) {
- bkc = new SimpleBookKeeperClient(getConf(), getUri());
- }
- printMetadata(dlm);
- } finally {
- dlm.close();
- if (null != bkc) {
- bkc.close();
- }
- }
- return 0;
- }
-
- private void printMetadata(DistributedLogManager dlm) throws Exception {
- printHeader(dlm);
- if (listSegments) {
- System.out.println("Ledgers : ");
- List<LogSegmentMetadata> segments = dlm.getLogSegments();
- for (LogSegmentMetadata segment : segments) {
- if (include(segment)) {
- printLedgerRow(segment);
- }
- }
- }
- }
-
- private void printHeader(DistributedLogManager dlm) throws Exception {
- DLSN firstDlsn = Await.result(dlm.getFirstDLSNAsync());
- boolean endOfStreamMarked = dlm.isEndOfStreamMarked();
- DLSN lastDlsn = dlm.getLastDLSN();
- long firstTxnId = dlm.getFirstTxId();
- long lastTxnId = dlm.getLastTxId();
- long recordCount = dlm.getLogRecordCount();
- String result = String.format("Stream : (firstTxId=%d, lastTxid=%d, firstDlsn=%s, lastDlsn=%s, endOfStreamMarked=%b, recordCount=%d)",
- firstTxnId, lastTxnId, getDlsnName(firstDlsn), getDlsnName(lastDlsn), endOfStreamMarked, recordCount);
- System.out.println(result);
- if (listEppStats) {
- printEppStatsHeader(dlm);
- }
- }
-
- boolean include(LogSegmentMetadata segment) {
- return (firstLid <= segment.getLogSegmentSequenceNumber() && (lastLid == -1 || lastLid >= segment.getLogSegmentSequenceNumber()));
- }
-
- private void printEppStatsHeader(DistributedLogManager dlm) throws Exception {
- String label = "Ledger Placement :";
- System.out.println(label);
- Map<BookieSocketAddress, Integer> totals = new HashMap<BookieSocketAddress, Integer>();
- List<LogSegmentMetadata> segments = dlm.getLogSegments();
- for (LogSegmentMetadata segment : segments) {
- if (include(segment)) {
- merge(totals, getBookieStats(segment));
- }
- }
- List<Map.Entry<BookieSocketAddress, Integer>> entries = new ArrayList<Map.Entry<BookieSocketAddress, Integer>>(totals.entrySet());
- Collections.sort(entries, new Comparator<Map.Entry<BookieSocketAddress, Integer>>() {
- @Override
- public int compare(Map.Entry<BookieSocketAddress, Integer> o1, Map.Entry<BookieSocketAddress, Integer> o2) {
- return o2.getValue() - o1.getValue();
- }
- });
- int width = 0;
- int totalEntries = 0;
- for (Map.Entry<BookieSocketAddress, Integer> entry : entries) {
- width = Math.max(width, label.length() + 1 + entry.getKey().toString().length());
- totalEntries += entry.getValue();
- }
- for (Map.Entry<BookieSocketAddress, Integer> entry : entries) {
- System.out.println(String.format("%"+width+"s\t%6.2f%%\t\t%d", entry.getKey(), entry.getValue()*1.0/totalEntries, entry.getValue()));
- }
- }
-
- private void printLedgerRow(LogSegmentMetadata segment) throws Exception {
- System.out.println(segment.getLogSegmentSequenceNumber() + "\t: " + segment);
- }
-
- private Map<BookieSocketAddress, Integer> getBookieStats(LogSegmentMetadata segment) throws Exception {
- Map<BookieSocketAddress, Integer> stats = new HashMap<BookieSocketAddress, Integer>();
- LedgerHandle lh = bkc.client().get().openLedgerNoRecovery(segment.getLogSegmentId(), BookKeeper.DigestType.CRC32,
- getConf().getBKDigestPW().getBytes(UTF_8));
- long eidFirst = 0;
- for (SortedMap.Entry<Long, ArrayList<BookieSocketAddress>> entry : LedgerReader.bookiesForLedger(lh).entrySet()) {
- long eidLast = entry.getKey().longValue();
- long count = eidLast - eidFirst + 1;
- for (BookieSocketAddress bookie : entry.getValue()) {
- merge(stats, bookie, (int) count);
- }
- eidFirst = eidLast;
- }
- return stats;
- }
-
- void merge(Map<BookieSocketAddress, Integer> m, BookieSocketAddress bookie, Integer count) {
- if (m.containsKey(bookie)) {
- m.put(bookie, count + m.get(bookie).intValue());
- } else {
- m.put(bookie, count);
- }
- }
-
- void merge(Map<BookieSocketAddress, Integer> m1, Map<BookieSocketAddress, Integer> m2) {
- for (Map.Entry<BookieSocketAddress, Integer> entry : m2.entrySet()) {
- merge(m1, entry.getKey(), entry.getValue());
- }
- }
-
- String getDlsnName(DLSN dlsn) {
- if (dlsn.equals(DLSN.InvalidDLSN)) {
- return "InvalidDLSN";
- }
- return dlsn.toString();
- }
-
- @Override
- protected String getUsage() {
- return "show [options]";
- }
- }
-
- static class CountCommand extends PerStreamCommand {
-
- DLSN startDLSN = null;
- DLSN endDLSN = null;
-
- protected CountCommand() {
- super("count", "count number records between dlsns");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- String[] args = cmdline.getArgs();
- if (args.length < 1) {
- throw new ParseException("Must specify at least start dlsn.");
- }
- if (args.length >= 1) {
- startDLSN = parseDLSN(args[0]);
- }
- if (args.length >= 2) {
- endDLSN = parseDLSN(args[1]);
- }
- }
-
- @Override
- protected int runCmd() throws Exception {
- DistributedLogManager dlm = getNamespace().openLog(getStreamName());
- try {
- long count = 0;
- if (null == endDLSN) {
- count = countToLastRecord(dlm);
- } else {
- count = countFromStartToEnd(dlm);
- }
- System.out.println("total is " + count + " records.");
- return 0;
- } finally {
- dlm.close();
- }
- }
-
- int countFromStartToEnd(DistributedLogManager dlm) throws Exception {
- int count = 0;
- try {
- LogReader reader = dlm.getInputStream(startDLSN);
- try {
- LogRecordWithDLSN record = reader.readNext(false);
- LogRecordWithDLSN preRecord = record;
- System.out.println("first record : " + record);
- while (null != record) {
- if (record.getDlsn().compareTo(endDLSN) > 0) {
- break;
- }
- ++count;
- if (count % 1000 == 0) {
- logger.info("read {} records from {}...", count, getStreamName());
- }
- preRecord = record;
- record = reader.readNext(false);
- }
- System.out.println("last record : " + preRecord);
- } finally {
- reader.close();
- }
- } finally {
- dlm.close();
- }
- return count;
- }
-
- long countToLastRecord(DistributedLogManager dlm) throws Exception {
- return Await.result(dlm.getLogRecordCountAsync(startDLSN)).longValue();
- }
-
- @Override
- protected String getUsage() {
- return "count <start> <end>";
- }
- }
-
- public static class DeleteCommand extends PerStreamCommand {
-
- protected DeleteCommand() {
- super("delete", "delete a given stream");
- }
-
- @Override
- protected int runCmd() throws Exception {
- getConf().setZkAclId(getZkAclId());
- DistributedLogManager dlm = getNamespace().openLog(getStreamName());
- try {
- dlm.delete();
- } finally {
- dlm.close();
- }
- return 0;
- }
-
- @Override
- protected String getUsage() {
- return "delete";
- }
- }
-
- public static class DeleteLedgersCommand extends PerDLCommand {
-
- private final List<Long> ledgers = new ArrayList<Long>();
-
- int numThreads = 1;
-
- protected DeleteLedgersCommand() {
- super("delete_ledgers", "delete given ledgers");
- options.addOption("l", "ledgers", true, "List of ledgers, separated by comma");
- options.addOption("lf", "ledgers-file", true, "File of list of ledgers, each line has a ledger id");
- options.addOption("t", "concurrency", true, "Number of threads to run deletions");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- if (cmdline.hasOption("l") && cmdline.hasOption("lf")) {
- throw new ParseException("Please specify ledgers: either use list or use file only.");
- }
- if (!cmdline.hasOption("l") && !cmdline.hasOption("lf")) {
- throw new ParseException("No ledgers specified. Please specify ledgers either use list or use file only.");
- }
- if (cmdline.hasOption("l")) {
- String ledgersStr = cmdline.getOptionValue("l");
- String[] ledgerStrs = ledgersStr.split(",");
- for (String ledgerStr : ledgerStrs) {
- ledgers.add(Long.parseLong(ledgerStr));
- }
- }
- if (cmdline.hasOption("lf")) {
- BufferedReader br = null;
- try {
-
- br = new BufferedReader(new InputStreamReader(
- new FileInputStream(new File(cmdline.getOptionValue("lf"))), UTF_8.name()));
- String line;
- while ((line = br.readLine()) != null) {
- ledgers.add(Long.parseLong(line));
- }
- } catch (FileNotFoundException e) {
- throw new ParseException("No ledgers file " + cmdline.getOptionValue("lf") + " found.");
- } catch (IOException e) {
- throw new ParseException("Invalid ledgers file " + cmdline.getOptionValue("lf") + " found.");
- } finally {
- if (null != br) {
- try {
- br.close();
- } catch (IOException e) {
- // no-op
- }
- }
- }
- }
- if (cmdline.hasOption("t")) {
- numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
- }
- }
-
- @Override
- protected String getUsage() {
- return "delete_ledgers [options]";
- }
-
- @Override
- protected int runCmd() throws Exception {
- ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
- try {
- final AtomicInteger numLedgers = new AtomicInteger(0);
- final CountDownLatch doneLatch = new CountDownLatch(numThreads);
- final AtomicInteger numFailures = new AtomicInteger(0);
- final LinkedBlockingQueue<Long> ledgerQueue =
- new LinkedBlockingQueue<Long>();
- ledgerQueue.addAll(ledgers);
- for (int i = 0; i < numThreads; i++) {
- final int tid = i;
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- while (true) {
- Long ledger = ledgerQueue.poll();
- if (null == ledger) {
- break;
- }
- try {
- getBookKeeperClient().get().deleteLedger(ledger);
- int numLedgersDeleted = numLedgers.incrementAndGet();
- if (numLedgersDeleted % 1000 == 0) {
- System.out.println("Deleted " + numLedgersDeleted + " ledgers.");
- }
- } catch (BKException.BKNoSuchLedgerExistsException e) {
- int numLedgersDeleted = numLedgers.incrementAndGet();
- if (numLedgersDeleted % 1000 == 0) {
- System.out.println("Deleted " + numLedgersDeleted + " ledgers.");
- }
- } catch (Exception e) {
- numFailures.incrementAndGet();
- break;
- }
- }
- doneLatch.countDown();
- System.out.println("Thread " + tid + " quits");
- }
- });
- }
- doneLatch.await();
- if (numFailures.get() > 0) {
- throw new IOException("Encounter " + numFailures.get() + " failures during deleting ledgers");
- }
- } finally {
- executorService.shutdown();
- }
- return 0;
- }
- }
-
- public static class CreateCommand extends PerDLCommand {
-
- final List<String> streams = new ArrayList<String>();
-
- String streamPrefix = null;
- String streamExpression = null;
-
- CreateCommand() {
- super("create", "create streams under a given namespace");
- options.addOption("r", "prefix", true, "Prefix of stream name. E.g. 'QuantumLeapTest-'.");
- options.addOption("e", "expression", true, "Expression to generate stream suffix. " +
- "Currently we support range 'x-y', list 'x,y,z' and name 'xyz'");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- if (cmdline.hasOption("r")) {
- streamPrefix = cmdline.getOptionValue("r");
- }
- if (cmdline.hasOption("e")) {
- streamExpression = cmdline.getOptionValue("e");
- }
- if (null == streamPrefix || null == streamExpression) {
- throw new ParseException("Please specify stream prefix & expression.");
- }
- }
-
- protected void generateStreams(String streamPrefix, String streamExpression) throws ParseException {
- // parse the stream expression
- if (streamExpression.contains("-")) {
- // a range expression
- String[] parts = streamExpression.split("-");
- if (parts.length != 2) {
- throw new ParseException("Invalid stream index range : " + streamExpression);
- }
- try {
- int start = Integer.parseInt(parts[0]);
- int end = Integer.parseInt(parts[1]);
- if (start > end) {
- throw new ParseException("Invalid stream index range : " + streamExpression);
- }
- for (int i = start; i <= end; i++) {
- streams.add(streamPrefix + i);
- }
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid stream index range : " + streamExpression);
- }
- } else if (streamExpression.contains(",")) {
- // a list expression
- String[] parts = streamExpression.split(",");
- try {
- for (String part : parts) {
- int idx = Integer.parseInt(part);
- streams.add(streamPrefix + idx);
- }
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid stream suffix list : " + streamExpression);
- }
- } else {
- streams.add(streamPrefix + streamExpression);
- }
- }
-
- @Override
- protected int runCmd() throws Exception {
- generateStreams(streamPrefix, streamExpression);
- if (streams.isEmpty()) {
- System.out.println("Nothing to create.");
- return 0;
- }
- if (!getForce() && !IOUtils.confirmPrompt("You are going to create streams : " + streams)) {
- return 0;
- }
- getConf().setZkAclId(getZkAclId());
- for (String stream : streams) {
- getNamespace().createLog(stream);
- }
- return 0;
- }
-
- @Override
- protected String getUsage() {
- return "create [options]";
- }
-
- protected void setPrefix(String prefix) {
- this.streamPrefix = prefix;
- }
-
- protected void setExpression(String expression) {
- this.streamExpression = expression;
- }
- }
-
- protected static class DumpCommand extends PerStreamCommand {
-
- boolean printHex = false;
- boolean skipPayload = false;
- Long fromTxnId = null;
- DLSN fromDLSN = null;
- int count = 100;
-
- DumpCommand() {
- super("dump", "dump records of a given stream");
- options.addOption("x", "hex", false, "Print record in hex format");
- options.addOption("sp", "skip-payload", false, "Skip printing the payload of the record");
- options.addOption("o", "offset", true, "Txn ID to start dumping.");
- options.addOption("n", "seqno", true, "Sequence Number to start dumping");
- options.addOption("e", "eid", true, "Entry ID to start dumping");
- options.addOption("t", "slot", true, "Slot to start dumping");
- options.addOption("l", "limit", true, "Number of entries to dump. Default is 100.");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- printHex = cmdline.hasOption("x");
- skipPayload = cmdline.hasOption("sp");
- if (cmdline.hasOption("o")) {
- try {
- fromTxnId = Long.parseLong(cmdline.getOptionValue("o"));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid txn id " + cmdline.getOptionValue("o"));
- }
- }
- if (cmdline.hasOption("l")) {
- try {
- count = Integer.parseInt(cmdline.getOptionValue("l"));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid count " + cmdline.getOptionValue("l"));
- }
- if (count <= 0) {
- throw new ParseException("Negative count found : " + count);
- }
- }
- if (cmdline.hasOption("n")) {
- long seqno;
- try {
- seqno = Long.parseLong(cmdline.getOptionValue("n"));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid sequence number " + cmdline.getOptionValue("n"));
- }
- long eid;
- if (cmdline.hasOption("e")) {
- eid = Long.parseLong(cmdline.getOptionValue("e"));
- } else {
- eid = 0;
- }
- long slot;
- if (cmdline.hasOption("t")) {
- slot = Long.parseLong(cmdline.getOptionValue("t"));
- } else {
- slot = 0;
- }
- fromDLSN = new DLSN(seqno, eid, slot);
- }
- if (null == fromTxnId && null == fromDLSN) {
- throw new ParseException("No start Txn/DLSN is specified.");
- }
- }
-
- @Override
- protected int runCmd() throws Exception {
- DistributedLogManager dlm = getNamespace().openLog(getStreamName());
- long totalCount = dlm.getLogRecordCount();
- try {
- AsyncLogReader reader;
- Object startOffset;
- try {
- DLSN lastDLSN = Await.result(dlm.getLastDLSNAsync());
- System.out.println("Last DLSN : " + lastDLSN);
- if (null == fromDLSN) {
- reader = dlm.getAsyncLogReader(fromTxnId);
- startOffset = fromTxnId;
- } else {
- reader = dlm.getAsyncLogReader(fromDLSN);
- startOffset = fromDLSN;
- }
- } catch (LogNotFoundException lee) {
- System.out.println("No stream found to dump records.");
- return 0;
- }
- try {
- System.out.println(String.format("Dump records for %s (from = %s, dump count = %d, total records = %d)",
- getStreamName(), startOffset, count, totalCount));
-
- dumpRecords(reader);
- } finally {
- Utils.close(reader);
- }
- } finally {
- dlm.close();
- }
- return 0;
- }
-
- private void dumpRecords(AsyncLogReader reader) throws Exception {
- int numRead = 0;
- LogRecord record = Await.result(reader.readNext());
- while (record != null) {
- // dump the record
- dumpRecord(record);
- ++numRead;
- if (numRead >= count) {
- break;
- }
- record = Await.result(reader.readNext());
- }
- if (numRead == 0) {
- System.out.println("No records.");
- } else {
- System.out.println("------------------------------------------------");
- }
- }
-
- private void dumpRecord(LogRecord record) {
- System.out.println("------------------------------------------------");
- if (record instanceof LogRecordWithDLSN) {
- System.out.println("Record (txn = " + record.getTransactionId() + ", bytes = "
- + record.getPayload().length + ", dlsn = "
- + ((LogRecordWithDLSN) record).getDlsn() + ", sequence id = "
- + ((LogRecordWithDLSN) record).getSequenceId() + ")");
- } else {
- System.out.println("Record (txn = " + record.getTransactionId() + ", bytes = "
- + record.getPayload().length + ")");
- }
- System.out.println("");
-
- if (skipPayload) {
- return;
- }
-
- if (printHex) {
- System.out.println(Hex.encodeHexString(record.getPayload()));
- } else {
- System.out.println(new String(record.getPayload(), UTF_8));
- }
- }
-
- @Override
- protected String getUsage() {
- return "dump [options]";
- }
-
- protected void setFromTxnId(Long fromTxnId) {
- this.fromTxnId = fromTxnId;
- }
- }
-
- /**
- * TODO: refactor inspect & inspectstream
- * TODO: support force
- *
- * inspectstream -lac -gap (different options for different operations for a single stream)
- * inspect -lac -gap (inspect the namespace, which will use inspect stream)
- */
- static class InspectStreamCommand extends PerStreamCommand {
-
- InspectStreamCommand() {
- super("inspectstream", "Inspect a given stream to identify any metadata corruptions");
- }
-
- @Override
- protected int runCmd() throws Exception {
- DistributedLogManager dlm = getNamespace().openLog(getStreamName());
- try {
- return inspectAndRepair(dlm.getLogSegments());
- } finally {
- dlm.close();
- }
- }
-
- protected int inspectAndRepair(List<LogSegmentMetadata> segments) throws Exception {
- LogSegmentMetadataStore metadataStore = getLogSegmentMetadataStore();
- ZooKeeperClient zkc = getZooKeeperClient();
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, getUri());
- BKDLConfig.propagateConfiguration(bkdlConfig, getConf());
- BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
- .dlConfig(getConf())
- .zkServers(bkdlConfig.getBkZkServersForReader())
- .ledgersPath(bkdlConfig.getBkLedgersPath())
- .name("dlog")
- .build();
- try {
- List<LogSegmentMetadata> segmentsToRepair = inspectLogSegments(bkc, segments);
- if (segmentsToRepair.isEmpty()) {
- System.out.println("The stream is good. No log segments to repair.");
- return 0;
- }
- System.out.println(segmentsToRepair.size() + " segments to repair : ");
- System.out.println(segmentsToRepair);
- System.out.println();
- if (!IOUtils.confirmPrompt("Do you want to repair them (Y/N): ")) {
- return 0;
- }
- repairLogSegments(metadataStore, bkc, segmentsToRepair);
- return 0;
- } finally {
- bkc.close();
- }
- }
-
- protected List<LogSegmentMetadata> inspectLogSegments(
- BookKeeperClient bkc, List<LogSegmentMetadata> segments) throws Exception {
- List<LogSegmentMetadata> segmentsToRepair = new ArrayList<LogSegmentMetadata>();
- for (LogSegmentMetadata segment : segments) {
- if (!segment.isInProgress() && !inspectLogSegment(bkc, segment)) {
- segmentsToRepair.add(segment);
- }
- }
- return segmentsToRepair;
- }
-
- /**
- * Inspect a given log segment.
- *
- * @param bkc
- * bookkeeper client
- * @param metadata
- * metadata of the log segment to
- * @return true if it is a good stream, false if the stream has inconsistent metadata.
- * @throws Exception
- */
- protected boolean inspectLogSegment(BookKeeperClient bkc,
- LogSegmentMetadata metadata) throws Exception {
- if (metadata.isInProgress()) {
- System.out.println("Skip inprogress log segment " + metadata);
- return true;
- }
- long ledgerId = metadata.getLogSegmentId();
- LedgerHandle lh = bkc.get().openLedger(ledgerId, BookKeeper.DigestType.CRC32,
- getConf().getBKDigestPW().getBytes(UTF_8));
- LedgerHandle readLh = bkc.get().openLedger(ledgerId, BookKeeper.DigestType.CRC32,
- getConf().getBKDigestPW().getBytes(UTF_8));
- LedgerReader lr = new LedgerReader(bkc.get());
- final AtomicReference<List<LedgerEntry>> entriesHolder = new AtomicReference<List<LedgerEntry>>(null);
- final AtomicInteger rcHolder = new AtomicInteger(-1234);
- final CountDownLatch doneLatch = new CountDownLatch(1);
- try {
- lr.forwardReadEntriesFromLastConfirmed(readLh, new BookkeeperInternalCallbacks.GenericCallback<List<LedgerEntry>>() {
- @Override
- public void operationComplete(int rc, List<LedgerEntry> entries) {
- rcHolder.set(rc);
- entriesHolder.set(entries);
- doneLatch.countDown();
- }
- });
- doneLatch.await();
- if (BKException.Code.OK != rcHolder.get()) {
- throw BKException.create(rcHolder.get());
- }
- List<LedgerEntry> entries = entriesHolder.get();
- long lastEntryId;
- if (entries.isEmpty()) {
- lastEntryId = LedgerHandle.INVALID_ENTRY_ID;
- } else {
- LedgerEntry lastEntry = entries.get(entries.size() - 1);
- lastEntryId = lastEntry.getEntryId();
- }
- if (lastEntryId != lh.getLastAddConfirmed()) {
- System.out.println("Inconsistent Last Add Confirmed Found for LogSegment " + metadata.getLogSegmentSequenceNumber() + ": ");
- System.out.println("\t metadata: " + metadata);
- System.out.println("\t lac in ledger metadata is " + lh.getLastAddConfirmed() + ", but lac in bookies is " + lastEntryId);
- return false;
- } else {
- return true;
- }
- } finally {
- lh.close();
- readLh.close();
- }
- }
-
- protected void repairLogSegments(LogSegmentMetadataStore metadataStore,
- BookKeeperClient bkc,
- List<LogSegmentMetadata> segments) throws Exception {
- BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
- try {
- MetadataUpdater metadataUpdater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(
- getConf(), metadataStore);
- for (LogSegmentMetadata segment : segments) {
- repairLogSegment(bkAdmin, metadataUpdater, segment);
- }
- } finally {
- bkAdmin.close();
- }
- }
-
- protected void repairLogSegment(BookKeeperAdmin bkAdmin,
- MetadataUpdater metadataUpdater,
- LogSegmentMetadata segment) throws Exception {
- if (segment.isInProgress()) {
- System.out.println("Skip inprogress log segment " + segment);
- return;
- }
- LedgerHandle lh = bkAdmin.openLedger(segment.getLogSegmentId(), true);
- long lac = lh.getLastAddConfirmed();
- Enumeration<LedgerEntry> entries = lh.readEntries(lac, lac);
- if (!entries.hasMoreElements()) {
- throw new IOException("Entry " + lac + " isn't found for " + segment);
- }
- LedgerEntry lastEntry = entries.nextElement();
- Entry.Reader reader = Entry.newBuilder()
- .setLogSegmentInfo(segment.getLogSegmentSequenceNumber(), segment.getStartSequenceId())
- .setEntryId(lastEntry.getEntryId())
- .setEnvelopeEntry(LogSegmentMetadata.supportsEnvelopedEntries(segment.getVersion()))
- .setInputStream(lastEntry.getEntryInputStream())
- .buildReader();
- LogRecordWithDLSN record = reader.nextRecord();
- LogRecordWithDLSN lastRecord = null;
- while (null != record) {
- lastRecord = record;
- record = reader.nextRecord();
- }
- if (null == lastRecord) {
- throw new IOException("No record found in entry " + lac + " for " + segment);
- }
- System.out.println("Updating last record for " + segment + " to " + lastRecord);
- if (!IOUtils.confirmPrompt("Do you want to make this change (Y/N): ")) {
- return;
- }
- metadataUpdater.updateLastRecord(segment, lastRecord);
- }
-
- @Override
- protected String getUsage() {
- return "inspectstream [options]";
- }
- }
-
- static interface BKCommandRunner {
- int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception;
- }
-
- abstract static class PerBKCommand extends PerDLCommand {
-
- protected PerBKCommand(String name, String description) {
- super(name, description);
- }
-
- @Override
- protected int runCmd() throws Exception {
- return runBKCommand(new BKCommandRunner() {
- @Override
- public int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
- return runBKCmd(zkc, bkc);
- }
- });
- }
-
- protected int runBKCommand(BKCommandRunner runner) throws Exception {
- return runner.run(getZooKeeperClient(), getBookKeeperClient());
- }
-
- abstract protected int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception;
- }
-
- static class RecoverCommand extends PerBKCommand {
-
- final List<Long> ledgers = new ArrayList<Long>();
- boolean query = false;
- boolean dryrun = false;
- boolean skipOpenLedgers = false;
- boolean fenceOnly = false;
- int fenceRate = 1;
- int concurrency = 1;
- final Set<BookieSocketAddress> bookiesSrc = new HashSet<BookieSocketAddress>();
- int partition = 0;
- int numPartitions = 0;
-
- RecoverCommand() {
- super("recover", "Recover the ledger data that stored on failed bookies");
- options.addOption("l", "ledger", true, "Specific ledger to recover");
- options.addOption("lf", "ledgerfile", true, "File contains ledgers list");
- options.addOption("q", "query", false, "Query the ledgers that contain given bookies");
- options.addOption("d", "dryrun", false, "Print the recovery plan w/o actually recovering");
- options.addOption("cy", "concurrency", true, "Number of ledgers could be recovered in parallel");
- options.addOption("sk", "skipOpenLedgers", false, "Skip recovering open ledgers");
- options.addOption("p", "partition", true, "partition");
- options.addOption("n", "num-partitions", true, "num partitions");
- options.addOption("fo", "fence-only", true, "fence the ledgers only w/o re-replicating entries");
- options.addOption("fr", "fence-rate", true, "rate on fencing ledgers");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- query = cmdline.hasOption("q");
- force = cmdline.hasOption("f");
- dryrun = cmdline.hasOption("d");
- skipOpenLedgers = cmdline.hasOption("sk");
- fenceOnly = cmdline.hasOption("fo");
- if (cmdline.hasOption("l")) {
- String[] lidStrs = cmdline.getOptionValue("l").split(",");
- try {
- for (String lidStr : lidStrs) {
- ledgers.add(Long.parseLong(lidStr));
- }
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid ledger id provided : " + cmdline.getOptionValue("l"));
- }
- }
- if (cmdline.hasOption("lf")) {
- String file = cmdline.getOptionValue("lf");
- try {
- BufferedReader br = new BufferedReader(
- new InputStreamReader(new FileInputStream(file), UTF_8.name()));
- try {
- String line = br.readLine();
-
- while (line != null) {
- ledgers.add(Long.parseLong(line));
- line = br.readLine();
- }
- } finally {
- br.close();
- }
- } catch (IOException e) {
- throw new ParseException("Invalid ledgers file provided : " + file);
- }
- }
- if (cmdline.hasOption("cy")) {
- try {
- concurrency = Integer.parseInt(cmdline.getOptionValue("cy"));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid concurrency provided : " + cmdline.getOptionValue("cy"));
- }
- }
- if (cmdline.hasOption("p")) {
- partition = Integer.parseInt(cmdline.getOptionValue("p"));
- }
- if (cmdline.hasOption("n")) {
- numPartitions = Integer.parseInt(cmdline.getOptionValue("n"));
- }
- if (cmdline.hasOption("fr")) {
- fenceRate = Integer.parseInt(cmdline.getOptionValue("fr"));
- }
- // Get bookies list to recover
- String[] args = cmdline.getArgs();
- final String[] bookieStrs = args[0].split(",");
- for (String bookieStr : bookieStrs) {
- final String bookieStrParts[] = bookieStr.split(":");
- if (bookieStrParts.length != 2) {
- throw new ParseException("BookieSrcs has invalid bookie address format (host:port expected) : "
- + bookieStr);
- }
- try {
- bookiesSrc.add(new BookieSocketAddress(bookieStrParts[0],
- Integer.parseInt(bookieStrParts[1])));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid ledger id provided : " + cmdline.getOptionValue("l"));
- }
- }
- }
-
- @Override
- protected int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
- BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
- try {
- if (query) {
- return bkQuery(bkAdmin, bookiesSrc);
- }
- if (fenceOnly) {
- return bkFence(bkc, ledgers, fenceRate);
- }
- if (!force) {
- System.out.println("Bookies : " + bookiesSrc);
- if (!IOUtils.confirmPrompt("Do you want to recover them: (Y/N)")) {
- return -1;
- }
- }
- if (!ledgers.isEmpty()) {
- System.out.println("Ledgers : " + ledgers);
- long numProcessed = 0;
- Iterator<Long> ledgersIter = ledgers.iterator();
- LinkedBlockingQueue<Long> ledgersToProcess = new LinkedBlockingQueue<Long>();
- while (ledgersIter.hasNext()) {
- long lid = ledgersIter.next();
- if (numPartitions <=0 || (numPartitions > 0 && lid % numPartitions == partition)) {
- ledgersToProcess.add(lid);
- ++numProcessed;
- }
- if (ledgersToProcess.size() == 10000) {
- System.out.println("Processing " + numProcessed + " ledgers");
- bkRecovery(ledgersToProcess, bookiesSrc, dryrun, skipOpenLedgers);
- ledgersToProcess.clear();
- System.out.println("Processed " + numProcessed + " ledgers");
- }
- }
- if (!ledgersToProcess.isEmpty()) {
- System.out.println("Processing " + numProcessed + " ledgers");
- bkRecovery(ledgersToProcess, bookiesSrc, dryrun, skipOpenLedgers);
- System.out.println("Processed " + numProcessed + " ledgers");
- }
- System.out.println("Done.");
- CountDownLatch latch = new CountDownLatch(1);
- latch.await();
- return 0;
- }
- return bkRecovery(bkAdmin, bookiesSrc, dryrun, skipOpenLedgers);
- } finally {
- bkAdmin.close();
- }
- }
-
- private int bkFence(final BookKeeperClient bkc, List<Long> ledgers, int fenceRate) throws Exception {
- if (ledgers.isEmpty()) {
- System.out.println("Nothing to fence. Done.");
- return 0;
- }
- ExecutorService executorService = Executors.newCachedThreadPool();
- final RateLimiter rateLimiter = RateLimiter.create(fenceRate);
- final byte[] passwd = getConf().getBKDigestPW().getBytes(UTF_8);
- final CountDownLatch latch = new CountDownLatch(ledgers.size());
- final AtomicInteger numPendings = new AtomicInteger(ledgers.size());
- final LinkedBlockingQueue<Long> ledgersQueue = new LinkedBlockingQueue<Long>();
- ledgersQueue.addAll(ledgers);
-
- for (int i = 0; i < concurrency; i++) {
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- while (!ledgersQueue.isEmpty()) {
- rateLimiter.acquire();
- Long lid = ledgersQueue.poll();
- if (null == lid) {
- break;
- }
- System.out.println("Fencing ledger " + lid);
- int numRetries = 3;
- while (numRetries > 0) {
- try {
- LedgerHandle lh = bkc.get().openLedger(lid, BookKeeper.DigestType.CRC32, passwd);
- lh.close();
- System.out.println("Fenced ledger " + lid + ", " + numPendings.decrementAndGet() + " left.");
- latch.countDown();
- } catch (BKException.BKNoSuchLedgerExistsException bke) {
- System.out.println("Skipped fence non-exist ledger " + lid + ", " + numPendings.decrementAndGet() + " left.");
- latch.countDown();
- } catch (BKException.BKLedgerRecoveryException lre) {
- --numRetries;
- continue;
- } catch (Exception e) {
- e.printStackTrace();
- break;
- }
- numRetries = 0;
- }
- }
- System.out.println("Thread exits");
- }
- });
- }
- latch.await();
- SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
- return 0;
- }
-
- private int bkQuery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddrs)
- throws InterruptedException, BKException {
- SortedMap<Long, LedgerMetadata> ledgersContainBookies =
- bkAdmin.getLedgersContainBookies(bookieAddrs);
- System.err.println("NOTE: Bookies in inspection list are marked with '*'.");
- for (Map.Entry<Long, LedgerMetadata> ledger : ledgersContainBookies.entrySet()) {
- System.out.println("ledger " + ledger.getKey() + " : " + ledger.getValue().getState());
- Map<Long, Integer> numBookiesToReplacePerEnsemble =
- inspectLedger(ledger.getValue(), bookieAddrs);
- System.out.print("summary: [");
- for (Map.Entry<Long, Integer> entry : numBookiesToReplacePerEnsemble.entrySet()) {
- System.out.print(entry.getKey() + "=" + entry.getValue() + ", ");
- }
- System.out.println("]");
- System.out.println();
- }
- System.out.println("Done");
- return 0;
- }
-
- private Map<Long, Integer> inspectLedger(LedgerMetadata metadata, Set<BookieSocketAddress> bookiesToInspect) {
- Map<Long, Integer> numBookiesToReplacePerEnsemble = new TreeMap<Long, Integer>();
- for (Map.Entry<Long, ArrayList<BookieSocketAddress>> ensemble : metadata.getEnsembles().entrySet()) {
- ArrayList<BookieSocketAddress> bookieList = ensemble.getValue();
- System.out.print(ensemble.getKey() + ":\t");
- int numBookiesToReplace = 0;
- for (BookieSocketAddress bookie: bookieList) {
- System.out.print(bookie.toString());
- if (bookiesToInspect.contains(bookie)) {
- System.out.print("*");
- ++numBookiesToReplace;
- } else {
- System.out.print(" ");
- }
- System.out.print(" ");
- }
- System.out.println();
- numBookiesToReplacePerEnsemble.put(ensemble.getKey(), numBookiesToReplace);
- }
- return numBookiesToReplacePerEnsemble;
- }
-
- private int bkRecovery(final LinkedBlockingQueue<Long> ledgers, final Set<BookieSocketAddress> bookieAddrs,
- final boolean dryrun, final boolean skipOpenLedgers)
- throws Exception {
- return runBKCommand(new BKCommandRunner() {
- @Override
- public int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
- BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
- try {
- bkRecovery(bkAdmin, ledgers, bookieAddrs, dryrun, skipOpenLedgers);
- return 0;
- } finally {
- bkAdmin.close();
- }
- }
- });
- }
-
- private int bkRecovery(final BookKeeperAdmin bkAdmin, final LinkedBlockingQueue<Long> ledgers,
- final Set<BookieSocketAddress> bookieAddrs,
- final boolean dryrun, final boolean skipOpenLedgers)
- throws InterruptedException, BKException {
- final AtomicInteger numPendings = new AtomicInteger(ledgers.size());
- final ExecutorService executorService = Executors.newCachedThreadPool();
- final CountDownLatch doneLatch = new CountDownLatch(concurrency);
- Runnable r = new Runnable() {
- @Override
- public void run() {
- while (!ledgers.isEmpty()) {
- long lid = -1L;
-
<TRUNCATED>
[07/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
new file mode 100644
index 0000000..a7b17f4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
@@ -0,0 +1,289 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Optional;
+import org.apache.distributedlog.ZooKeeperClient.Credentials;
+import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.net.NetUtils;
+import org.apache.distributedlog.util.ConfUtils;
+import com.twitter.util.Future;
+import com.twitter.util.Promise;
+import com.twitter.util.Return;
+import com.twitter.util.Throw;
+import org.apache.bookkeeper.client.AsyncCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.net.DNSToSwitchMapping;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.zookeeper.KeeperException;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * BookKeeper Client wrapper over {@link BookKeeper}.
+ *
+ * <h3>Metrics</h3>
+ * <ul>
+ * <li> bookkeeper operation stats are exposed under current scope by {@link BookKeeper}
+ * </ul>
+ */
+public class BookKeeperClient {
+ static final Logger LOG = LoggerFactory.getLogger(BookKeeperClient.class);
+
+ // Parameters to build bookkeeper client
+ private final DistributedLogConfiguration conf;
+ private final String name;
+ private final String zkServers;
+ private final String ledgersPath;
+ private final byte[] passwd;
+ private final ClientSocketChannelFactory channelFactory;
+ private final HashedWheelTimer requestTimer;
+ private final StatsLogger statsLogger;
+
+ // bookkeeper client state
+ private boolean closed = false;
+ private BookKeeper bkc = null;
+ private ZooKeeperClient zkc;
+ private final boolean ownZK;
+ // feature provider
+ private final Optional<FeatureProvider> featureProvider;
+
+ @SuppressWarnings("deprecation")
+ private synchronized void commonInitialization(
+ DistributedLogConfiguration conf, String ledgersPath,
+ ClientSocketChannelFactory channelFactory, StatsLogger statsLogger, HashedWheelTimer requestTimer)
+ throws IOException, InterruptedException, KeeperException {
+ ClientConfiguration bkConfig = new ClientConfiguration();
+ bkConfig.setAddEntryTimeout(conf.getBKClientWriteTimeout());
+ bkConfig.setReadTimeout(conf.getBKClientReadTimeout());
+ bkConfig.setZkLedgersRootPath(ledgersPath);
+ bkConfig.setZkTimeout(conf.getBKClientZKSessionTimeoutMilliSeconds());
+ bkConfig.setNumWorkerThreads(conf.getBKClientNumberWorkerThreads());
+ bkConfig.setEnsemblePlacementPolicy(RegionAwareEnsemblePlacementPolicy.class);
+ bkConfig.setZkRequestRateLimit(conf.getBKClientZKRequestRateLimit());
+ bkConfig.setProperty(RegionAwareEnsemblePlacementPolicy.REPP_DISALLOW_BOOKIE_PLACEMENT_IN_REGION_FEATURE_NAME,
+ DistributedLogConstants.DISALLOW_PLACEMENT_IN_REGION_FEATURE_NAME);
+ // reload configuration from dl configuration with settings prefixed with 'bkc.'
+ ConfUtils.loadConfiguration(bkConfig, conf, "bkc.");
+
+ Class<? extends DNSToSwitchMapping> dnsResolverCls;
+ try {
+ dnsResolverCls = conf.getEnsemblePlacementDnsResolverClass();
+ } catch (ConfigurationException e) {
+ LOG.error("Failed to load bk dns resolver : ", e);
+ throw new IOException("Failed to load bk dns resolver : ", e);
+ }
+ final DNSToSwitchMapping dnsResolver =
+ NetUtils.getDNSResolver(dnsResolverCls, conf.getBkDNSResolverOverrides());
+
+ this.bkc = BookKeeper.newBuilder()
+ .config(bkConfig)
+ .zk(zkc.get())
+ .channelFactory(channelFactory)
+ .statsLogger(statsLogger)
+ .dnsResolver(dnsResolver)
+ .requestTimer(requestTimer)
+ .featureProvider(featureProvider.orNull())
+ .build();
+ }
+
+ BookKeeperClient(DistributedLogConfiguration conf,
+ String name,
+ String zkServers,
+ ZooKeeperClient zkc,
+ String ledgersPath,
+ ClientSocketChannelFactory channelFactory,
+ HashedWheelTimer requestTimer,
+ StatsLogger statsLogger,
+ Optional<FeatureProvider> featureProvider) {
+ this.conf = conf;
+ this.name = name;
+ this.zkServers = zkServers;
+ this.ledgersPath = ledgersPath;
+ this.passwd = conf.getBKDigestPW().getBytes(UTF_8);
+ this.channelFactory = channelFactory;
+ this.requestTimer = requestTimer;
+ this.statsLogger = statsLogger;
+ this.featureProvider = featureProvider;
+ this.ownZK = null == zkc;
+ if (null != zkc) {
+ // reference the passing zookeeper client
+ this.zkc = zkc;
+ }
+ }
+
+ private synchronized void initialize() throws IOException {
+ if (null != this.bkc) {
+ return;
+ }
+ if (null == this.zkc) {
+ int zkSessionTimeout = conf.getBKClientZKSessionTimeoutMilliSeconds();
+ RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
+ conf.getBKClientZKRetryBackoffStartMillis(),
+ conf.getBKClientZKRetryBackoffMaxMillis(), conf.getBKClientZKNumRetries());
+ Credentials credentials = Credentials.NONE;
+ if (conf.getZkAclId() != null) {
+ credentials = new DigestCredentials(conf.getZkAclId(), conf.getZkAclId());
+ }
+
+ this.zkc = new ZooKeeperClient(name + ":zk", zkSessionTimeout, 2 * zkSessionTimeout, zkServers,
+ retryPolicy, statsLogger.scope("bkc_zkc"), conf.getZKClientNumberRetryThreads(),
+ conf.getBKClientZKRequestRateLimit(), credentials);
+ }
+
+ try {
+ commonInitialization(conf, ledgersPath, channelFactory, statsLogger, requestTimer);
+ } catch (InterruptedException e) {
+ throw new DLInterruptedException("Interrupted on creating bookkeeper client " + name + " : ", e);
+ } catch (KeeperException e) {
+ throw new ZKException("Error on creating bookkeeper client " + name + " : ", e);
+ }
+
+ if (ownZK) {
+ LOG.info("BookKeeper Client created {} with its own ZK Client : ledgersPath = {}, numRetries = {}, " +
+ "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
+ new Object[] { name, ledgersPath,
+ conf.getBKClientZKNumRetries(), conf.getBKClientZKSessionTimeoutMilliSeconds(),
+ conf.getBKClientZKRetryBackoffStartMillis(), conf.getBKClientZKRetryBackoffMaxMillis(),
+ conf.getBkDNSResolverOverrides() });
+ } else {
+ LOG.info("BookKeeper Client created {} with shared zookeeper client : ledgersPath = {}, numRetries = {}, " +
+ "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
+ new Object[] { name, ledgersPath,
+ conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(),
+ conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(),
+ conf.getBkDNSResolverOverrides() });
+ }
+ }
+
+
+ public synchronized BookKeeper get() throws IOException {
+ checkClosedOrInError();
+ if (null == bkc) {
+ initialize();
+ }
+ return bkc;
+ }
+
+ // Util functions
+ public Future<LedgerHandle> createLedger(int ensembleSize,
+ int writeQuorumSize,
+ int ackQuorumSize) {
+ BookKeeper bk;
+ try {
+ bk = get();
+ } catch (IOException ioe) {
+ return Future.exception(ioe);
+ }
+ final Promise<LedgerHandle> promise = new Promise<LedgerHandle>();
+ bk.asyncCreateLedger(ensembleSize, writeQuorumSize, ackQuorumSize,
+ BookKeeper.DigestType.CRC32, passwd, new AsyncCallback.CreateCallback() {
+ @Override
+ public void createComplete(int rc, LedgerHandle lh, Object ctx) {
+ if (BKException.Code.OK == rc) {
+ promise.updateIfEmpty(new Return<LedgerHandle>(lh));
+ } else {
+ promise.updateIfEmpty(new Throw<LedgerHandle>(BKException.create(rc)));
+ }
+ }
+ }, null);
+ return promise;
+ }
+
+ public Future<Void> deleteLedger(long lid,
+ final boolean ignoreNonExistentLedger) {
+ BookKeeper bk;
+ try {
+ bk = get();
+ } catch (IOException ioe) {
+ return Future.exception(ioe);
+ }
+ final Promise<Void> promise = new Promise<Void>();
+ bk.asyncDeleteLedger(lid, new AsyncCallback.DeleteCallback() {
+ @Override
+ public void deleteComplete(int rc, Object ctx) {
+ if (BKException.Code.OK == rc) {
+ promise.updateIfEmpty(new Return<Void>(null));
+ } else if (BKException.Code.NoSuchLedgerExistsException == rc) {
+ if (ignoreNonExistentLedger) {
+ promise.updateIfEmpty(new Return<Void>(null));
+ } else {
+ promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
+ }
+ } else {
+ promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
+ }
+ }
+ }, null);
+ return promise;
+ }
+
+ 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 != bkcToClose) {
+ try {
+ bkcToClose.close();
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted on closing bookkeeper client {} : ", name, e);
+ Thread.currentThread().interrupt();
+ } catch (BKException e) {
+ LOG.warn("Error on closing bookkeeper client {} : ", name, e);
+ }
+ }
+ if (null != zkcToClose) {
+ if (ownZK) {
+ zkcToClose.close();
+ }
+ }
+ }
+
+ public synchronized void checkClosedOrInError() throws AlreadyClosedException {
+ if (closed) {
+ LOG.error("BookKeeper Client {} is already closed", name);
+ throw new AlreadyClosedException("BookKeeper Client " + name + " is already closed");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
new file mode 100644
index 0000000..a356f9f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
@@ -0,0 +1,209 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+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.apache.bookkeeper.feature.FeatureProvider;
+
+import org.apache.bookkeeper.feature.Feature;
+
+/**
+ * Builder to build bookkeeper client.
+ */
+public class BookKeeperClientBuilder {
+
+ /**
+ * Create a bookkeeper client builder to build bookkeeper clients.
+ *
+ * @return bookkeeper client builder.
+ */
+ public static BookKeeperClientBuilder newBuilder() {
+ return new BookKeeperClientBuilder();
+ }
+
+ // client name
+ private String name = null;
+ // dl config
+ private DistributedLogConfiguration dlConfig = null;
+ // bookkeeper settings
+ // zookeeper client
+ private ZooKeeperClient zkc = null;
+ // or zookeeper servers
+ private String zkServers = null;
+ // ledgers path
+ private String ledgersPath = null;
+ // statsLogger
+ private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+ // client channel factory
+ private ClientSocketChannelFactory channelFactory = null;
+ // request timer
+ private HashedWheelTimer requestTimer = null;
+ // feature provider
+ private Optional<FeatureProvider> featureProvider = Optional.absent();
+
+ // Cached BookKeeper Client
+ private BookKeeperClient cachedClient = null;
+
+ /**
+ * Private bookkeeper builder.
+ */
+ private BookKeeperClientBuilder() {}
+
+ /**
+ * Set client name.
+ *
+ * @param name
+ * client name.
+ * @return builder
+ */
+ public synchronized BookKeeperClientBuilder name(String name) {
+ this.name = name;
+ return this;
+ }
+
+ /**
+ * <i>dlConfig</i> used to configure bookkeeper client.
+ *
+ * @param dlConfig
+ * distributedlog config.
+ * @return builder.
+ */
+ public synchronized BookKeeperClientBuilder dlConfig(DistributedLogConfiguration dlConfig) {
+ this.dlConfig = dlConfig;
+ return this;
+ }
+
+ /**
+ * Set the zkc used to build bookkeeper client. If a zookeeper client is provided in this
+ * method, bookkeeper client will use it rather than creating a brand new one.
+ *
+ * @param zkc
+ * zookeeper client.
+ * @return builder
+ * @see #zkServers(String)
+ */
+ public synchronized BookKeeperClientBuilder zkc(ZooKeeperClient zkc) {
+ this.zkc = zkc;
+ return this;
+ }
+
+ /**
+ * Set the zookeeper servers that bookkeeper client would connect to. If no zookeeper client
+ * is provided by {@link #zkc(ZooKeeperClient)}, bookkeeper client will use the given string
+ * to create a brand new zookeeper client.
+ *
+ * @param zkServers
+ * zookeeper servers that bookkeeper client would connect to.
+ * @return builder
+ * @see #zkc(ZooKeeperClient)
+ */
+ public synchronized BookKeeperClientBuilder zkServers(String zkServers) {
+ this.zkServers = zkServers;
+ return this;
+ }
+
+ /**
+ * Set the ledgers path that bookkeeper client is going to access.
+ *
+ * @param ledgersPath
+ * ledgers path
+ * @return builder
+ * @see org.apache.bookkeeper.conf.ClientConfiguration#getZkLedgersRootPath()
+ */
+ public synchronized BookKeeperClientBuilder ledgersPath(String ledgersPath) {
+ this.ledgersPath = ledgersPath;
+ return this;
+ }
+
+ /**
+ * Build BookKeeper client using existing <i>bkc</i> client.
+ *
+ * @param bkc
+ * bookkeeper client.
+ * @return builder
+ */
+ public synchronized BookKeeperClientBuilder bkc(BookKeeperClient bkc) {
+ this.cachedClient = bkc;
+ return this;
+ }
+
+ /**
+ * Build BookKeeper client using existing <i>channelFactory</i>.
+ *
+ * @param channelFactory
+ * Channel Factory used to build bookkeeper client.
+ * @return bookkeeper client builder.
+ */
+ public synchronized BookKeeperClientBuilder channelFactory(ClientSocketChannelFactory channelFactory) {
+ this.channelFactory = channelFactory;
+ return this;
+ }
+
+ /**
+ * Build BookKeeper client using existing <i>request timer</i>.
+ *
+ * @param requestTimer
+ * HashedWheelTimer used to build bookkeeper client.
+ * @return bookkeeper client builder.
+ */
+ public synchronized BookKeeperClientBuilder requestTimer(HashedWheelTimer requestTimer) {
+ this.requestTimer = requestTimer;
+ return this;
+ }
+
+ /**
+ * Build BookKeeper Client using given stats logger <i>statsLogger</i>.
+ *
+ * @param statsLogger
+ * stats logger to report stats
+ * @return builder.
+ */
+ public synchronized BookKeeperClientBuilder statsLogger(StatsLogger statsLogger) {
+ this.statsLogger = statsLogger;
+ return this;
+ }
+
+ public synchronized BookKeeperClientBuilder featureProvider(Optional<FeatureProvider> featureProvider) {
+ this.featureProvider = featureProvider;
+ return this;
+ }
+
+ private void validateParameters() {
+ Preconditions.checkNotNull(name, "Missing client name.");
+ Preconditions.checkNotNull(dlConfig, "Missing DistributedLog Configuration.");
+ Preconditions.checkArgument(null == zkc || null == zkServers, "Missing zookeeper setting.");
+ Preconditions.checkNotNull(ledgersPath, "Missing Ledgers Root Path.");
+ }
+
+ public synchronized BookKeeperClient build() {
+ if (null == cachedClient) {
+ cachedClient = buildClient();
+ }
+ return cachedClient;
+ }
+
+ private BookKeeperClient buildClient() {
+ validateParameters();
+ return new BookKeeperClient(dlConfig, name, zkServers, zkc, ledgersPath, channelFactory, requestTimer, statsLogger, featureProvider);
+ }
+}
[34/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogSegmentWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogSegmentWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogSegmentWriter.java
deleted file mode 100644
index df64505..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogSegmentWriter.java
+++ /dev/null
@@ -1,1348 +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 java.io.IOException;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantLock;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.FlushException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException;
-import com.twitter.distributedlog.exceptions.WriteCancelledException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.exceptions.InvalidEnvelopedEntryException;
-import com.twitter.distributedlog.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.injector.FailureInjector;
-import com.twitter.distributedlog.injector.RandomDelayFailureInjector;
-import com.twitter.distributedlog.io.Buffer;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.io.CompressionUtils;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
-import com.twitter.distributedlog.logsegment.LogSegmentWriter;
-import com.twitter.distributedlog.stats.BroadCastStatsLogger;
-import com.twitter.distributedlog.stats.OpStatsListener;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.SafeQueueingFuturePool;
-import com.twitter.distributedlog.util.SimplePermitLimiter;
-import com.twitter.distributedlog.util.Sizable;
-import com.twitter.util.Function0;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.FuturePool;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
-import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.MathUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
-
-/**
- * BookKeeper Based Log Segment Writer.
- *
- * Multiple log records are packed into a single bookkeeper
- * entry before sending it over the network. The fact that the log record entries
- * are complete in the bookkeeper entries means that each bookkeeper log entry
- * can be read as a complete edit log. This is useful for reading, as we don't
- * need to read through the entire log segment to get the last written entry.
- *
- * <h3>Metrics</h3>
- *
- * <ul>
- * <li> flush/periodic/{success,miss}: counters for periodic flushes.
- * <li> data/{success,miss}: counters for data transmits.
- * <li> transmit/packetsize: opstats. characteristics of packet size for transmits.
- * <li> control/success: counter of success transmit of control records
- * <li> seg_writer/write: opstats. latency characteristics of write operations in segment writer.
- * <li> seg_writer/add_complete/{callback,queued,deferred}: opstats. latency components of add completions.
- * <li> seg_writer/pendings: counter. the number of records pending by the segment writers.
- * <li> transmit/outstanding/requests: per stream gauge. the number of outstanding transmits each stream.
- * </ul>
- */
-class BKLogSegmentWriter implements LogSegmentWriter, AddCallback, Runnable, Sizable {
- static final Logger LOG = LoggerFactory.getLogger(BKLogSegmentWriter.class);
-
- private final String fullyQualifiedLogSegment;
- private final String streamName;
- private final int logSegmentMetadataVersion;
- private BKTransmitPacket packetPrevious;
- private Entry.Writer recordSetWriter;
- private final AtomicInteger outstandingTransmits;
- private final int transmissionThreshold;
- protected final LogSegmentEntryWriter entryWriter;
- private final CompressionCodec.Type compressionType;
- private final ReentrantLock transmitLock = new ReentrantLock();
- private final AtomicInteger transmitResult
- = new AtomicInteger(BKException.Code.OK);
- private final DistributedLock lock;
- private final boolean isDurableWriteEnabled;
- private DLSN lastDLSN = DLSN.InvalidDLSN;
- private final long startTxId;
- private long lastTxId = DistributedLogConstants.INVALID_TXID;
- private long lastTxIdAcknowledged = DistributedLogConstants.INVALID_TXID;
- private long outstandingBytes = 0;
- private long numFlushesSinceRestart = 0;
- private long numBytes = 0;
- private long lastEntryId = Long.MIN_VALUE;
- private long lastTransmitNanos = Long.MIN_VALUE;
- private final int periodicKeepAliveMs;
-
- // Indicates whether there are writes that have been successfully transmitted that would need
- // a control record to be transmitted to make them visible to the readers by updating the last
- // add confirmed
- volatile private boolean controlFlushNeeded = false;
- private boolean immediateFlushEnabled = false;
- private int minDelayBetweenImmediateFlushMs = 0;
- private Stopwatch lastTransmit;
- private boolean streamEnded = false;
- private final ScheduledFuture<?> periodicFlushSchedule;
- private final ScheduledFuture<?> periodicKeepAliveSchedule;
- final private AtomicReference<ScheduledFuture<?>> transmitSchedFutureRef = new AtomicReference<ScheduledFuture<?>>(null);
- final private AtomicReference<ScheduledFuture<?>> immFlushSchedFutureRef = new AtomicReference<ScheduledFuture<?>>(null);
- final private AtomicReference<Exception> scheduledFlushException = new AtomicReference<Exception>(null);
- private boolean enforceLock = true;
- private Promise<Void> closeFuture = null;
- private final boolean enableRecordCounts;
- private int positionWithinLogSegment = 0;
- private final long logSegmentSequenceNumber;
- // Used only for values that *could* change (e.g. buffer size etc.)
- private final DistributedLogConfiguration conf;
- private final OrderedScheduler scheduler;
-
- // stats
- private final StatsLogger envelopeStatsLogger;
- private final StatsLogger transmitOutstandingLogger;
- private final Counter transmitDataSuccesses;
- private final Counter transmitDataMisses;
- private final Gauge<Number> transmitOutstandingGauge;
- private final OpStatsLogger transmitDataPacketSize;
- private final Counter transmitControlSuccesses;
- private final Counter pFlushSuccesses;
- private final Counter pFlushMisses;
- private final OpStatsLogger writeTime;
- private final OpStatsLogger addCompleteTime;
- private final OpStatsLogger addCompleteQueuedTime;
- private final OpStatsLogger addCompleteDeferredTime;
- private final Counter pendingWrites;
-
- // add complete processing
- private final SafeQueueingFuturePool<Void> addCompleteFuturePool;
-
- // Functions
- private final AbstractFunction1<Integer, Future<Long>> GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC =
- new AbstractFunction1<Integer, Future<Long>>() {
- @Override
- public Future<Long> apply(Integer transmitRc) {
- if (BKException.Code.OK == transmitRc) {
- return Future.value(getLastTxIdAcknowledged());
- } else {
- return Future.exception(new BKTransmitException("Failed to transmit entry", transmitRc));
- }
- }
- };
- final AbstractFunction1<Long, Future<Long>> COMMIT_AFTER_FLUSH_FUNC =
- new AbstractFunction1<Long, Future<Long>>() {
- @Override
- public Future<Long> apply(Long lastAckedTxId) {
- return commit();
- }
- };
-
- private final AlertStatsLogger alertStatsLogger;
- private final WriteLimiter writeLimiter;
- private final FailureInjector writeDelayInjector;
-
- /**
- * Construct an edit log output stream which writes to a ledger.
- */
- protected BKLogSegmentWriter(String streamName,
- String logSegmentName,
- DistributedLogConfiguration conf,
- int logSegmentMetadataVersion,
- LogSegmentEntryWriter entryWriter,
- DistributedLock lock, /** the lock needs to be acquired **/
- long startTxId,
- long logSegmentSequenceNumber,
- OrderedScheduler scheduler,
- StatsLogger statsLogger,
- StatsLogger perLogStatsLogger,
- AlertStatsLogger alertStatsLogger,
- PermitLimiter globalWriteLimiter,
- FeatureProvider featureProvider,
- DynamicDistributedLogConfiguration dynConf)
- throws IOException {
- super();
-
- // set up a write limiter
- PermitLimiter streamWriteLimiter = null;
- if (conf.getPerWriterOutstandingWriteLimit() < 0) {
- streamWriteLimiter = PermitLimiter.NULL_PERMIT_LIMITER;
- } else {
- Feature disableWriteLimitFeature = featureProvider.getFeature(
- CoreFeatureKeys.DISABLE_WRITE_LIMIT.name().toLowerCase());
- streamWriteLimiter = new SimplePermitLimiter(
- conf.getOutstandingWriteLimitDarkmode(),
- conf.getPerWriterOutstandingWriteLimit(),
- statsLogger.scope("streamWriteLimiter"),
- false,
- disableWriteLimitFeature);
- }
- this.writeLimiter = new WriteLimiter(streamName, streamWriteLimiter, globalWriteLimiter);
- this.alertStatsLogger = alertStatsLogger;
- this.envelopeStatsLogger = BroadCastStatsLogger.masterslave(statsLogger, perLogStatsLogger);
-
- StatsLogger flushStatsLogger = statsLogger.scope("flush");
- StatsLogger pFlushStatsLogger = flushStatsLogger.scope("periodic");
- pFlushSuccesses = pFlushStatsLogger.getCounter("success");
- pFlushMisses = pFlushStatsLogger.getCounter("miss");
-
- // transmit
- StatsLogger transmitDataStatsLogger = statsLogger.scope("data");
- transmitDataSuccesses = transmitDataStatsLogger.getCounter("success");
- transmitDataMisses = transmitDataStatsLogger.getCounter("miss");
- StatsLogger transmitStatsLogger = statsLogger.scope("transmit");
- transmitDataPacketSize = transmitStatsLogger.getOpStatsLogger("packetsize");
- StatsLogger transmitControlStatsLogger = statsLogger.scope("control");
- transmitControlSuccesses = transmitControlStatsLogger.getCounter("success");
- StatsLogger segWriterStatsLogger = statsLogger.scope("seg_writer");
- writeTime = segWriterStatsLogger.getOpStatsLogger("write");
- addCompleteTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("callback");
- addCompleteQueuedTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("queued");
- addCompleteDeferredTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("deferred");
- pendingWrites = segWriterStatsLogger.getCounter("pending");
-
- // outstanding transmit requests
- transmitOutstandingLogger = perLogStatsLogger.scope("transmit").scope("outstanding");
- transmitOutstandingGauge = new Gauge<Number>() {
- @Override
- public Number getDefaultValue() {
- return 0;
- }
- @Override
- public Number getSample() {
- return outstandingTransmits.get();
- }
- };
- transmitOutstandingLogger.registerGauge("requests", transmitOutstandingGauge);
-
- outstandingTransmits = new AtomicInteger(0);
- this.fullyQualifiedLogSegment = streamName + ":" + logSegmentName;
- this.streamName = streamName;
- this.logSegmentMetadataVersion = logSegmentMetadataVersion;
- this.entryWriter = entryWriter;
- this.lock = lock;
- this.lock.checkOwnershipAndReacquire();
-
- final int configuredTransmissionThreshold = dynConf.getOutputBufferSize();
- if (configuredTransmissionThreshold > MAX_LOGRECORDSET_SIZE) {
- LOG.warn("Setting output buffer size {} greater than max transmission size {} for log segment {}",
- new Object[] {configuredTransmissionThreshold, MAX_LOGRECORDSET_SIZE, fullyQualifiedLogSegment});
- this.transmissionThreshold = MAX_LOGRECORDSET_SIZE;
- } else {
- this.transmissionThreshold = configuredTransmissionThreshold;
- }
- this.compressionType = CompressionUtils.stringToType(conf.getCompressionType());
-
- this.logSegmentSequenceNumber = logSegmentSequenceNumber;
- this.recordSetWriter = Entry.newEntry(
- streamName,
- Math.max(transmissionThreshold, 1024),
- envelopeBeforeTransmit(),
- compressionType,
- envelopeStatsLogger);
- this.packetPrevious = null;
- this.startTxId = startTxId;
- this.lastTxId = startTxId;
- this.lastTxIdAcknowledged = startTxId;
- this.enableRecordCounts = conf.getEnableRecordCounts();
- this.immediateFlushEnabled = conf.getImmediateFlushEnabled();
- this.isDurableWriteEnabled = dynConf.isDurableWriteEnabled();
- this.scheduler = scheduler;
-
- // Failure injection
- if (conf.getEIInjectWriteDelay()) {
- this.writeDelayInjector = new RandomDelayFailureInjector(dynConf);
- } else {
- this.writeDelayInjector = FailureInjector.NULL;
- }
-
- // If we are transmitting immediately (threshold == 0) and if immediate
- // flush is enabled, we don't need the periodic flush task
- final int configuredPeriodicFlushFrequency = dynConf.getPeriodicFlushFrequencyMilliSeconds();
- if (!immediateFlushEnabled || (0 != this.transmissionThreshold)) {
- int periodicFlushFrequency = configuredPeriodicFlushFrequency;
- if (periodicFlushFrequency > 0 && scheduler != null) {
- periodicFlushSchedule = scheduler.scheduleAtFixedRate(this,
- periodicFlushFrequency/2, periodicFlushFrequency/2, TimeUnit.MILLISECONDS);
- } else {
- periodicFlushSchedule = null;
- }
- } else {
- // Min delay heuristic applies only when immediate flush is enabled
- // and transmission threshold is zero
- minDelayBetweenImmediateFlushMs = conf.getMinDelayBetweenImmediateFlushMs();
- periodicFlushSchedule = null;
- }
- this.periodicKeepAliveMs = conf.getPeriodicKeepAliveMilliSeconds();
- if (periodicKeepAliveMs > 0 && scheduler != null) {
- periodicKeepAliveSchedule = scheduler.scheduleAtFixedRate(new Runnable() {
- @Override
- public void run() {
- keepAlive();
- }
- }, periodicKeepAliveMs, periodicKeepAliveMs, TimeUnit.MILLISECONDS);
- } else {
- periodicKeepAliveSchedule = null;
- }
-
- this.conf = conf;
- if (null != scheduler) {
- this.addCompleteFuturePool = new SafeQueueingFuturePool<Void>(scheduler.getFuturePool(streamName));
- } else {
- this.addCompleteFuturePool = null;
- }
- assert(!this.immediateFlushEnabled || (null != this.scheduler));
- this.lastTransmit = Stopwatch.createStarted();
- }
-
- String getFullyQualifiedLogSegment() {
- return fullyQualifiedLogSegment;
- }
-
- @VisibleForTesting
- DistributedLock getLock() {
- return this.lock;
- }
-
- @VisibleForTesting
- FuturePool getFuturePool() {
- if (null == scheduler) {
- return null;
- }
- return scheduler.getFuturePool(streamName);
- }
-
- @VisibleForTesting
- void setTransmitResult(int rc) {
- transmitResult.set(rc);
- }
-
- @VisibleForTesting
- protected final LogSegmentEntryWriter getEntryWriter() {
- return this.entryWriter;
- }
-
- @Override
- public long getLogSegmentId() {
- return this.entryWriter.getLogSegmentId();
- }
-
- protected final long getLogSegmentSequenceNumber() {
- return logSegmentSequenceNumber;
- }
-
- /**
- * Get the start tx id of the log segment.
- *
- * @return start tx id of the log segment.
- */
- protected final long getStartTxId() {
- return startTxId;
- }
-
- /**
- * Get the last tx id that has been written to the log segment buffer but not committed yet.
- *
- * @return last tx id that has been written to the log segment buffer but not committed yet.
- * @see #getLastTxIdAcknowledged()
- */
- synchronized long getLastTxId() {
- return lastTxId;
- }
-
- /**
- * Get the last tx id that has been acknowledged.
- *
- * @return last tx id that has been acknowledged.
- * @see #getLastTxId()
- */
- synchronized long getLastTxIdAcknowledged() {
- return lastTxIdAcknowledged;
- }
-
- /**
- * Get the position-within-logsemgnet of the last written log record.
- *
- * @return position-within-logsegment of the last written log record.
- */
- synchronized int getPositionWithinLogSegment() {
- return positionWithinLogSegment;
- }
-
- @VisibleForTesting
- long getLastEntryId() {
- return lastEntryId;
- }
-
- /**
- * Get the last dlsn of the last acknowledged record.
- *
- * @return last dlsn of the last acknowledged record.
- */
- synchronized DLSN getLastDLSN() {
- return lastDLSN;
- }
-
- @Override
- public long size() {
- return entryWriter.size();
- }
-
- private synchronized int getAverageTransmitSize() {
- if (numFlushesSinceRestart > 0) {
- long ret = numBytes/numFlushesSinceRestart;
-
- if (ret < Integer.MIN_VALUE || ret > Integer.MAX_VALUE) {
- throw new IllegalArgumentException
- (ret + " transmit size should never exceed max transmit size");
- }
- return (int) ret;
- }
-
- return 0;
- }
-
- private Entry.Writer newRecordSetWriter() {
- return Entry.newEntry(
- streamName,
- Math.max(transmissionThreshold, getAverageTransmitSize()),
- envelopeBeforeTransmit(),
- compressionType,
- envelopeStatsLogger);
- }
-
- private boolean envelopeBeforeTransmit() {
- return LogSegmentMetadata.supportsEnvelopedEntries(logSegmentMetadataVersion);
- }
-
- @Override
- public Future<Void> asyncClose() {
- return closeInternal(false);
- }
-
- @Override
- public Future<Void> asyncAbort() {
- return closeInternal(true);
- }
-
- private void flushAddCompletes() {
- if (null != addCompleteFuturePool) {
- addCompleteFuturePool.close();
- }
- }
-
- private synchronized void abortPacket(BKTransmitPacket packet) {
- long numRecords = 0;
- if (null != packet) {
- EntryBuffer recordSet = packet.getRecordSet();
- numRecords = recordSet.getNumRecords();
- int rc = transmitResult.get();
- if (BKException.Code.OK == rc) {
- rc = BKException.Code.InterruptedException;
- }
- Throwable reason = new WriteCancelledException(streamName, FutureUtils.transmitException(rc));
- recordSet.abortTransmit(reason);
- }
- LOG.info("Stream {} aborted {} writes", fullyQualifiedLogSegment, numRecords);
- }
-
- private synchronized long getWritesPendingTransmit() {
- if (null != recordSetWriter) {
- return recordSetWriter.getNumRecords();
- } else {
- return 0;
- }
- }
-
- private synchronized long getPendingAddCompleteCount() {
- if (null != addCompleteFuturePool) {
- return addCompleteFuturePool.size();
- } else {
- return 0;
- }
- }
-
- private Future<Void> closeInternal(boolean abort) {
- Promise<Void> closePromise;
- synchronized (this) {
- if (null != closeFuture) {
- return closeFuture;
- }
- closePromise = closeFuture = new Promise<Void>();
- }
-
- AtomicReference<Throwable> throwExc = new AtomicReference<Throwable>(null);
- closeInternal(abort, throwExc, closePromise);
- return closePromise;
- }
-
- private void closeInternal(final boolean abort,
- final AtomicReference<Throwable> throwExc,
- final Promise<Void> closePromise) {
- // clean stats resources
- this.transmitOutstandingLogger.unregisterGauge("requests", transmitOutstandingGauge);
- this.writeLimiter.close();
-
- // Cancel the periodic keep alive schedule first
- if (null != periodicKeepAliveSchedule) {
- if (!periodicKeepAliveSchedule.cancel(false)) {
- LOG.info("Periodic keepalive for log segment {} isn't cancelled.", getFullyQualifiedLogSegment());
- }
- }
-
- // Cancel the periodic flush schedule first
- // The task is allowed to exit gracefully
- if (null != periodicFlushSchedule) {
- // we don't need to care about the cancel result here. if the periodicl flush task couldn't
- // be cancelled, it means that it is doing flushing. So following flushes would be synchronized
- // to wait until background flush completed.
- if (!periodicFlushSchedule.cancel(false)) {
- LOG.info("Periodic flush for log segment {} isn't cancelled.", getFullyQualifiedLogSegment());
- }
- }
-
- // If it is a normal close and the stream isn't in an error state, we attempt to flush any buffered data
- if (!abort && !isLogSegmentInError()) {
- this.enforceLock = false;
- LOG.info("Flushing before closing log segment {}", getFullyQualifiedLogSegment());
- flushAndCommit().addEventListener(new FutureEventListener<Long>() {
- @Override
- public void onSuccess(Long value) {
- abortTransmitPacketOnClose(abort, throwExc, closePromise);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- throwExc.set(cause);
- abortTransmitPacketOnClose(abort, throwExc, closePromise);
- }
- });
- } else {
- abortTransmitPacketOnClose(abort, throwExc, closePromise);
- }
-
- }
-
- private void abortTransmitPacketOnClose(final boolean abort,
- final AtomicReference<Throwable> throwExc,
- final Promise<Void> closePromise) {
- LOG.info("Closing BKPerStreamLogWriter (abort={}) for {} :" +
- " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {} addCompletesPending = {}",
- new Object[]{abort, fullyQualifiedLogSegment, getLastDLSN(),
- outstandingTransmits.get(), getWritesPendingTransmit(), getPendingAddCompleteCount()});
-
- // Save the current packet to reset, leave a new empty packet to avoid a race with
- // addCompleteDeferredProcessing.
- final BKTransmitPacket packetPreviousSaved;
- final BKTransmitPacket packetCurrentSaved;
- synchronized (this) {
- packetPreviousSaved = packetPrevious;
- packetCurrentSaved = new BKTransmitPacket(recordSetWriter);
- recordSetWriter = newRecordSetWriter();
- }
-
- // Once the last packet been transmitted, apply any remaining promises asynchronously
- // to avoid blocking close if bk client is slow for some reason.
- if (null != packetPreviousSaved) {
- packetPreviousSaved.addTransmitCompleteListener(new FutureEventListener<Integer>() {
- @Override
- public void onSuccess(Integer transmitResult) {
- flushAddCompletes();
- abortPacket(packetCurrentSaved);
- }
- @Override
- public void onFailure(Throwable cause) {
- LOG.error("Unexpected error on transmit completion ", cause);
- }
- });
- } else {
- // In this case there are no pending add completes, but we still need to abort the
- // current packet.
- abortPacket(packetCurrentSaved);
- }
- closeLedgerOnClose(abort, throwExc, closePromise);
- }
-
- private void closeLedgerOnClose(final boolean abort,
- final AtomicReference<Throwable> throwExc,
- final Promise<Void> closePromise) {
- // close the log segment if it isn't in error state, so all the outstanding addEntry(s) will callback.
- if (null == throwExc.get() && !isLogSegmentInError()) {
- // Synchronous closing the ledger handle, if we couldn't close a ledger handle successfully.
- // we should throw the exception to #closeToFinalize, so it would fail completing a log segment.
- entryWriter.asyncClose(new CloseCallback() {
- @Override
- public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
- if (BKException.Code.OK != rc && BKException.Code.LedgerClosedException != rc) {
- if (!abort) {
- throwExc.set(new IOException("Failed to close ledger for " + fullyQualifiedLogSegment + " : " +
- BKException.getMessage(rc)));
- }
- }
- completeClosePromise(abort, throwExc, closePromise);
- }
- }, null);
- } else {
- completeClosePromise(abort, throwExc, closePromise);
- }
- }
-
- private void completeClosePromise(final boolean abort,
- final AtomicReference<Throwable> throwExc,
- final Promise<Void> closePromise) {
- // If add entry failed because of closing ledger above, we don't need to fail the close operation
- if (!abort && null == throwExc.get() && shouldFailCompleteLogSegment()) {
- throwExc.set(new BKTransmitException("Closing an errored stream : ", transmitResult.get()));
- }
-
- if (null == throwExc.get()) {
- FutureUtils.setValue(closePromise, null);
- } else {
- FutureUtils.setException(closePromise, throwExc.get());
- }
- }
-
- @Override
- synchronized public void write(LogRecord record) throws IOException {
- writeUserRecord(record);
- flushIfNeeded();
- }
-
- @Override
- synchronized public Future<DLSN> asyncWrite(LogRecord record) {
- return asyncWrite(record, true);
- }
-
- synchronized public Future<DLSN> asyncWrite(LogRecord record, boolean flush) {
- Future<DLSN> result = null;
- try {
- if (record.isControl()) {
- // we don't pack control records with user records together
- // so transmit current output buffer if possible
- try {
- transmit();
- } catch (IOException ioe) {
- return Future.exception(new WriteCancelledException(fullyQualifiedLogSegment, ioe));
- }
- result = writeControlLogRecord(record);
- transmit();
- } else {
- result = writeUserRecord(record);
- if (!isDurableWriteEnabled) {
- // we have no idea about the DLSN if durability is turned off.
- result = Future.value(DLSN.InvalidDLSN);
- }
- if (flush) {
- flushIfNeeded();
- }
- }
- } catch (IOException ioe) {
- // We may incorrectly report transmit failure here, but only if we happened to hit
- // packet/xmit size limit conditions AND fail flush above, which should happen rarely
- if (null != result) {
- LOG.error("Overriding first result with flush failure {}", result);
- }
- result = Future.exception(ioe);
-
- // Flush to ensure any prev. writes with flush=false are flushed despite failure.
- flushIfNeededNoThrow();
- }
- return result;
- }
-
- synchronized private Future<DLSN> writeUserRecord(LogRecord record) throws IOException {
- if (null != closeFuture) {
- throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(BKException.Code.LedgerClosedException));
- }
-
- if (BKException.Code.OK != transmitResult.get()) {
- // Failfast if the stream already encountered error with safe retry on the client
- throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(transmitResult.get()));
- }
-
- if (streamEnded) {
- throw new EndOfStreamException("Writing to a stream after it has been marked as completed");
- }
-
- if ((record.getTransactionId() < 0) ||
- (record.getTransactionId() == DistributedLogConstants.MAX_TXID)) {
- throw new TransactionIdOutOfOrderException(record.getTransactionId());
- }
-
- // Inject write delay if configured to do so
- writeDelayInjector.inject();
-
- // Will check write rate limits and throw if exceeded.
- writeLimiter.acquire();
- pendingWrites.inc();
-
- // The count represents the number of user records up to the
- // current record
- // Increment the record count only when writing a user log record
- // Internally generated log records don't increment the count
- // writeInternal will always set a count regardless of whether it was
- // incremented or not.
- Future<DLSN> future = null;
- try {
- // increment the position for the record to write
- // if the record is failed to write, it would be decremented.
- positionWithinLogSegment++;
- int numRecords = 1;
- if (record.isRecordSet()) {
- numRecords = LogRecordSet.numRecords(record);
- }
- future = writeInternal(record);
- // after the record (record set) is written, the position should be
- // moved for {numRecords}, but since we already moved the record by 1
- // so advance the position for other {numRecords - 1}.
- positionWithinLogSegment += (numRecords - 1);
- } catch (IOException ex) {
- writeLimiter.release();
- pendingWrites.dec();
- positionWithinLogSegment--;
- throw ex;
- }
-
- // Track outstanding requests and return the future.
- return future.ensure(new Function0<BoxedUnit>() {
- public BoxedUnit apply() {
- pendingWrites.dec();
- writeLimiter.release();
- return null;
- }
- });
- }
-
- boolean isLogSegmentInError() {
- return (transmitResult.get() != BKException.Code.OK);
- }
-
- boolean shouldFailCompleteLogSegment() {
- return (transmitResult.get() != BKException.Code.OK) &&
- (transmitResult.get() != BKException.Code.LedgerClosedException);
- }
-
- synchronized public Future<DLSN> writeInternal(LogRecord record)
- throws LogRecordTooLongException, LockingException, BKTransmitException,
- WriteException, InvalidEnvelopedEntryException {
- int logRecordSize = record.getPersistentSize();
-
- if (logRecordSize > MAX_LOGRECORD_SIZE) {
- throw new LogRecordTooLongException(String.format(
- "Log Record of size %d written when only %d is allowed",
- logRecordSize, MAX_LOGRECORD_SIZE));
- }
-
- // If we will exceed the max number of bytes allowed per entry
- // initiate a transmit before accepting the new log record
- if ((recordSetWriter.getNumBytes() + logRecordSize) > MAX_LOGRECORDSET_SIZE) {
- checkStateAndTransmit();
- }
-
- checkWriteLock();
-
- if (enableRecordCounts) {
- // Set the count here. The caller would appropriately increment it
- // if this log record is to be counted
- record.setPositionWithinLogSegment(positionWithinLogSegment);
- }
-
- Promise<DLSN> writePromise = new Promise<DLSN>();
- writePromise.addEventListener(new OpStatsListener<DLSN>(writeTime));
- recordSetWriter.writeRecord(record, writePromise);
-
- if (record.getTransactionId() < lastTxId) {
- LOG.info("Log Segment {} TxId decreased Last: {} Record: {}",
- new Object[] {fullyQualifiedLogSegment, lastTxId, record.getTransactionId()});
- }
- if (!record.isControl()) {
- // only update last tx id for user records
- lastTxId = record.getTransactionId();
- outstandingBytes += (20 + record.getPayload().length);
- }
- return writePromise;
- }
-
- synchronized private Future<DLSN> writeControlLogRecord()
- throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
- LockingException, LogRecordTooLongException {
- LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.CONTROL_RECORD_CONTENT);
- controlRec.setControl();
- return writeControlLogRecord(controlRec);
- }
-
- synchronized private Future<DLSN> writeControlLogRecord(LogRecord record)
- throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
- LockingException, LogRecordTooLongException {
- return writeInternal(record);
- }
-
- /**
- * We write a special log record that marks the end of the stream. Since this is the last
- * log record in the stream, it is marked with MAX_TXID. MAX_TXID also has the useful
- * side-effect of disallowing future startLogSegment calls through the MaxTxID check
- *
- * @throws IOException
- */
- synchronized private void writeEndOfStreamMarker() throws IOException {
- LogRecord endOfStreamRec = new LogRecord(DistributedLogConstants.MAX_TXID, "endOfStream".getBytes(UTF_8));
- endOfStreamRec.setEndOfStream();
- writeInternal(endOfStreamRec);
- }
-
- /**
- * Flushes all the data up to this point,
- * adds the end of stream marker and marks the stream
- * as read-only in the metadata. No appends to the
- * stream will be allowed after this point
- */
- public Future<Long> markEndOfStream() {
- synchronized (this) {
- try {
- writeEndOfStreamMarker();
- } catch (IOException e) {
- return Future.exception(e);
- }
- streamEnded = true;
- }
- return flushAndCommit();
- }
-
- /**
- * Write bulk of records.
- *
- * (TODO: moved this method to log writer level)
- *
- * @param records list of records to write
- * @return number of records that has been written
- * @throws IOException when there is I/O errors during writing records.
- */
- synchronized public int writeBulk(List<LogRecord> records) throws IOException {
- int numRecords = 0;
- for (LogRecord r : records) {
- write(r);
- numRecords++;
- }
- return numRecords;
- }
-
- private void checkStateBeforeTransmit() throws WriteException {
- try {
- FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitBeforeAddEntry);
- } catch (IOException e) {
- throw new WriteException(streamName, "Fail transmit before adding entries");
- }
- }
-
- /**
- * Transmit the output buffer data to the backend.
- *
- * @return last txn id that already acknowledged
- * @throws BKTransmitException if the segment writer is already in error state
- * @throws LockingException if the segment writer lost lock before transmit
- * @throws WriteException if failed to create the envelope for the data to transmit
- * @throws InvalidEnvelopedEntryException when built an invalid enveloped entry
- */
- synchronized void checkStateAndTransmit()
- throws BKTransmitException, WriteException, InvalidEnvelopedEntryException, LockingException {
- checkStateBeforeTransmit();
- transmit();
- }
-
- @Override
- public synchronized Future<Long> flush() {
- try {
- checkStateBeforeTransmit();
- } catch (WriteException e) {
- return Future.exception(e);
- }
-
- Future<Integer> transmitFuture;
- try {
- transmitFuture = transmit();
- } catch (BKTransmitException e) {
- return Future.exception(e);
- } catch (LockingException e) {
- return Future.exception(e);
- } catch (WriteException e) {
- return Future.exception(e);
- } catch (InvalidEnvelopedEntryException e) {
- return Future.exception(e);
- }
-
- if (null == transmitFuture) {
- if (null != packetPrevious) {
- transmitFuture = packetPrevious.getTransmitFuture();
- } else {
- return Future.value(getLastTxIdAcknowledged());
- }
- }
-
- return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
- }
-
- @Override
- public synchronized Future<Long> commit() {
- // we don't pack control records with user records together
- // so transmit current output buffer if possible
- Future<Integer> transmitFuture;
- try {
- try {
- transmitFuture = transmit();
- } catch (IOException ioe) {
- return Future.exception(ioe);
- }
- if (null == transmitFuture) {
- writeControlLogRecord();
- return flush();
- }
- } catch (IOException ioe) {
- return Future.exception(ioe);
- }
- return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
- }
-
- Future<Long> flushAndCommit() {
- return flush().flatMap(COMMIT_AFTER_FLUSH_FUNC);
- }
-
- void flushIfNeededNoThrow() {
- try {
- flushIfNeeded();
- } catch (IOException ioe) {
- LOG.error("Encountered exception while flushing log records to stream {}",
- fullyQualifiedLogSegment, ioe);
- }
- }
-
- void scheduleFlushWithDelayIfNeeded(final Callable<?> callable,
- final AtomicReference<ScheduledFuture<?>> scheduledFutureRef) {
- final long delayMs = Math.max(0, minDelayBetweenImmediateFlushMs - lastTransmit.elapsed(TimeUnit.MILLISECONDS));
- final ScheduledFuture<?> scheduledFuture = scheduledFutureRef.get();
- if ((null == scheduledFuture) || scheduledFuture.isDone()) {
- scheduledFutureRef.set(scheduler.schedule(new Runnable() {
- @Override
- public void run() {
- synchronized(this) {
- scheduledFutureRef.set(null);
- try {
- callable.call();
-
- // Flush was successful or wasn't needed, the exception should be unset.
- scheduledFlushException.set(null);
- } catch (Exception exc) {
- scheduledFlushException.set(exc);
- LOG.error("Delayed flush failed", exc);
- }
- }
- }
- }, delayMs, TimeUnit.MILLISECONDS));
- }
- }
-
- // Based on transmit buffer size, immediate flush, etc., should we flush the current
- // packet now.
- void flushIfNeeded() throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
- LockingException, FlushException {
- if (outstandingBytes > transmissionThreshold) {
- // If flush delay is disabled, flush immediately, else schedule appropriately.
- if (0 == minDelayBetweenImmediateFlushMs) {
- checkStateAndTransmit();
- } else {
- scheduleFlushWithDelayIfNeeded(new Callable<Void>() {
- @Override
- public Void call() throws Exception {
- checkStateAndTransmit();
- return null;
- }
- }, transmitSchedFutureRef);
-
- // Timing here is not very important--the last flush failed and we should
- // indicate this to the caller. The next flush may succeed and unset the
- // scheduledFlushException in which case the next write will succeed (if the caller
- // hasn't already closed the writer).
- if (scheduledFlushException.get() != null) {
- throw new FlushException("Last flush encountered an error while writing data to the backend",
- getLastTxId(), getLastTxIdAcknowledged(), scheduledFlushException.get());
- }
- }
- }
- }
-
- private void checkWriteLock() throws LockingException {
- try {
- if (FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_WriteInternalLostLock)) {
- throw new LockingException("/failpoint/lockpath", "failpoint is simulating a lost lock"
- + getFullyQualifiedLogSegment());
- }
- } catch (IOException e) {
- throw new LockingException("/failpoint/lockpath", "failpoint is simulating a lost lock for "
- + getFullyQualifiedLogSegment());
- }
- if (enforceLock) {
- lock.checkOwnershipAndReacquire();
- }
- }
-
- /**
- * Transmit the current buffer to bookkeeper.
- * Synchronised at the class. #write() and #setReadyToFlush()
- * are never called at the same time.
- *
- * NOTE: This method should only throw known exceptions so that we don't accidentally
- * add new code that throws in an inappropriate place.
- *
- * @return a transmit future for caller to wait for transmit result if we transmit successfully,
- * null if no data to transmit
- * @throws BKTransmitException if the segment writer is already in error state
- * @throws LockingException if the segment writer lost lock before transmit
- * @throws WriteException if failed to create the envelope for the data to transmit
- * @throws InvalidEnvelopedEntryException when built an invalid enveloped entry
- */
- private Future<Integer> transmit()
- throws BKTransmitException, LockingException, WriteException, InvalidEnvelopedEntryException {
- EntryBuffer recordSetToTransmit;
- transmitLock.lock();
- try {
- synchronized (this) {
- checkWriteLock();
- // If transmitResult is anything other than BKException.Code.OK, it means that the
- // stream has encountered an error and cannot be written to.
- if (!transmitResult.compareAndSet(BKException.Code.OK,
- BKException.Code.OK)) {
- LOG.error("Log Segment {} Trying to write to an errored stream; Error is {}",
- fullyQualifiedLogSegment,
- BKException.getMessage(transmitResult.get()));
- throw new BKTransmitException("Trying to write to an errored stream;"
- + " Error code : (" + transmitResult.get()
- + ") " + BKException.getMessage(transmitResult.get()), transmitResult.get());
- }
-
- if (recordSetWriter.getNumRecords() == 0) {
- // Control flushes always have at least the control record to flush
- transmitDataMisses.inc();
- return null;
- }
-
- recordSetToTransmit = recordSetWriter;
- recordSetWriter = newRecordSetWriter();
- outstandingBytes = 0;
-
- if (recordSetToTransmit.hasUserRecords()) {
- numBytes += recordSetToTransmit.getNumBytes();
- numFlushesSinceRestart++;
- }
- }
-
- Buffer toSend;
- try {
- toSend = recordSetToTransmit.getBuffer();
- FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitFailGetBuffer);
- } catch (IOException e) {
- if (e instanceof InvalidEnvelopedEntryException) {
- alertStatsLogger.raise("Invalid enveloped entry for segment {} : ", fullyQualifiedLogSegment, e);
- }
- LOG.error("Exception while enveloping entries for segment: {}",
- new Object[] {fullyQualifiedLogSegment}, e);
- // If a write fails here, we need to set the transmit result to an error so that
- // no future writes go through and violate ordering guarantees.
- transmitResult.set(BKException.Code.WriteException);
- if (e instanceof InvalidEnvelopedEntryException) {
- alertStatsLogger.raise("Invalid enveloped entry for segment {} : ", fullyQualifiedLogSegment, e);
- throw (InvalidEnvelopedEntryException) e;
- } else {
- throw new WriteException(streamName, "Envelope Error");
- }
- }
-
- synchronized (this) {
- // update the transmit timestamp
- lastTransmitNanos = MathUtils.nowInNano();
-
- BKTransmitPacket packet = new BKTransmitPacket(recordSetToTransmit);
- packetPrevious = packet;
- entryWriter.asyncAddEntry(toSend.getData(), 0, toSend.size(),
- this, packet);
-
- if (recordSetToTransmit.hasUserRecords()) {
- transmitDataSuccesses.inc();
- } else {
- transmitControlSuccesses.inc();
- }
-
- lastTransmit.reset().start();
- outstandingTransmits.incrementAndGet();
- controlFlushNeeded = false;
- return packet.getTransmitFuture();
- }
- } finally {
- transmitLock.unlock();
- }
- }
-
- /**
- * Checks if there is any data to transmit so that the periodic flush
- * task can determine if there is anything it needs to do
- */
- synchronized private boolean haveDataToTransmit() {
- if (!transmitResult.compareAndSet(BKException.Code.OK, BKException.Code.OK)) {
- // Even if there is data it cannot be transmitted, so effectively nothing to send
- return false;
- }
-
- return (recordSetWriter.getNumRecords() > 0);
- }
-
- @Override
- public void addComplete(final int rc, LedgerHandle handle,
- final long entryId, final Object ctx) {
- final AtomicReference<Integer> effectiveRC = new AtomicReference<Integer>(rc);
- try {
- if (FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitComplete)) {
- effectiveRC.set(BKException.Code.UnexpectedConditionException);
- }
- } catch (Exception exc) {
- effectiveRC.set(BKException.Code.UnexpectedConditionException);
- }
-
- // Sanity check to make sure we're receiving these callbacks in order.
- if (entryId > -1 && lastEntryId >= entryId) {
- LOG.error("Log segment {} saw out of order entry {} lastEntryId {}",
- new Object[] {fullyQualifiedLogSegment, entryId, lastEntryId});
- }
- lastEntryId = entryId;
-
- assert (ctx instanceof BKTransmitPacket);
- final BKTransmitPacket transmitPacket = (BKTransmitPacket) ctx;
-
- // Time from transmit until receipt of addComplete callback
- addCompleteTime.registerSuccessfulEvent(TimeUnit.MICROSECONDS.convert(
- System.nanoTime() - transmitPacket.getTransmitTime(), TimeUnit.NANOSECONDS));
-
- if (BKException.Code.OK == rc) {
- EntryBuffer recordSet = transmitPacket.getRecordSet();
- if (recordSet.hasUserRecords()) {
- synchronized (this) {
- lastTxIdAcknowledged = Math.max(lastTxIdAcknowledged, recordSet.getMaxTxId());
- }
- }
- }
-
- if (null != addCompleteFuturePool) {
- final Stopwatch queuedTime = Stopwatch.createStarted();
- addCompleteFuturePool.apply(new Function0<Void>() {
- public Void apply() {
- final Stopwatch deferredTime = Stopwatch.createStarted();
- addCompleteQueuedTime.registerSuccessfulEvent(queuedTime.elapsed(TimeUnit.MICROSECONDS));
- addCompleteDeferredProcessing(transmitPacket, entryId, effectiveRC.get());
- addCompleteDeferredTime.registerSuccessfulEvent(deferredTime.elapsed(TimeUnit.MICROSECONDS));
- return null;
- }
- @Override
- public String toString() {
- return String.format("AddComplete(Stream=%s, entryId=%d, rc=%d)",
- fullyQualifiedLogSegment, entryId, rc);
- }
- }).addEventListener(new FutureEventListener<Void>() {
- @Override
- public void onSuccess(Void done) {
- }
- @Override
- public void onFailure(Throwable cause) {
- LOG.error("addComplete processing failed for {} entry {} lastTxId {} rc {} with error",
- new Object[] {fullyQualifiedLogSegment, entryId, transmitPacket.getRecordSet().getMaxTxId(), rc, cause});
- }
- });
- // Race condition if we notify before the addComplete is enqueued.
- transmitPacket.notifyTransmitComplete(effectiveRC.get());
- outstandingTransmits.getAndDecrement();
- } else {
- // Notify transmit complete must be called before deferred processing in the
- // sync case since otherwise callbacks in deferred processing may deadlock.
- transmitPacket.notifyTransmitComplete(effectiveRC.get());
- outstandingTransmits.getAndDecrement();
- addCompleteDeferredProcessing(transmitPacket, entryId, effectiveRC.get());
- }
- }
-
- private void addCompleteDeferredProcessing(final BKTransmitPacket transmitPacket,
- final long entryId,
- final int rc) {
- boolean cancelPendingPromises = false;
- EntryBuffer recordSet = transmitPacket.getRecordSet();
- synchronized (this) {
- if (transmitResult.compareAndSet(BKException.Code.OK, rc)) {
- // If this is the first time we are setting an error code in the transmitResult then
- // we must cancel pending promises; once this error has been set, more records will not
- // be enqueued; they will be failed with WriteException
- cancelPendingPromises = (BKException.Code.OK != rc);
- } else {
- LOG.warn("Log segment {} entryId {}: Tried to set transmit result to ({}) but is already ({})",
- new Object[] {fullyQualifiedLogSegment, entryId, rc, transmitResult.get()});
- }
-
- if (transmitResult.get() != BKException.Code.OK) {
- if (recordSet.hasUserRecords()) {
- transmitDataPacketSize.registerFailedEvent(recordSet.getNumBytes());
- }
- } else {
- // If we had data that we flushed then we need it to make sure that
- // background flush in the next pass will make the previous writes
- // visible by advancing the lastAck
- if (recordSet.hasUserRecords()) {
- transmitDataPacketSize.registerSuccessfulEvent(recordSet.getNumBytes());
- controlFlushNeeded = true;
- if (immediateFlushEnabled) {
- if (0 == minDelayBetweenImmediateFlushMs) {
- backgroundFlush(true);
- } else {
- scheduleFlushWithDelayIfNeeded(new Callable<Void>() {
- @Override
- public Void call() throws Exception {
- backgroundFlush(true);
- return null;
- }
- }, immFlushSchedFutureRef);
- }
- }
- }
- }
-
- // update last dlsn before satisifying future
- if (BKException.Code.OK == transmitResult.get()) {
- DLSN lastDLSNInPacket = recordSet.finalizeTransmit(
- logSegmentSequenceNumber, entryId);
- if (recordSet.hasUserRecords()) {
- if (null != lastDLSNInPacket && lastDLSN.compareTo(lastDLSNInPacket) < 0) {
- lastDLSN = lastDLSNInPacket;
- }
- }
- }
- }
-
- if (BKException.Code.OK == transmitResult.get()) {
- recordSet.completeTransmit(logSegmentSequenceNumber, entryId);
- } else {
- recordSet.abortTransmit(FutureUtils.transmitException(transmitResult.get()));
- }
-
- if (cancelPendingPromises) {
- // Since the writer is in a bad state no more packets will be tramsitted, and its safe to
- // assign a new empty packet. This is to avoid a race with closeInternal which may also
- // try to cancel the current packet;
- final BKTransmitPacket packetCurrentSaved;
- synchronized (this) {
- packetCurrentSaved = new BKTransmitPacket(recordSetWriter);
- recordSetWriter = newRecordSetWriter();
- }
- packetCurrentSaved.getRecordSet().abortTransmit(
- new WriteCancelledException(streamName,
- FutureUtils.transmitException(transmitResult.get())));
- }
- }
-
- @Override
- synchronized public void run() {
- backgroundFlush(false);
- }
-
- synchronized private void backgroundFlush(boolean controlFlushOnly) {
- if (null != closeFuture) {
- // if the log segment is closing, skip any background flushing
- LOG.debug("Skip background flushing since log segment {} is closing.", getFullyQualifiedLogSegment());
- return;
- }
- try {
- boolean newData = haveDataToTransmit();
-
- if (controlFlushNeeded || (!controlFlushOnly && newData)) {
- // If we need this periodic transmit to persist previously written data but
- // there is no new data (which would cause the transmit to be skipped) generate
- // a control record
- if (!newData) {
- writeControlLogRecord();
- }
-
- transmit();
- pFlushSuccesses.inc();
- } else {
- pFlushMisses.inc();
- }
- } catch (IOException exc) {
- LOG.error("Log Segment {}: Error encountered by the periodic flush", fullyQualifiedLogSegment, exc);
- }
- }
-
- synchronized private void keepAlive() {
- if (null != closeFuture) {
- // if the log segment is closing, skip sending any keep alive records.
- LOG.debug("Skip sending keepAlive control record since log segment {} is closing.",
- getFullyQualifiedLogSegment());
- return;
- }
-
- if (MathUtils.elapsedMSec(lastTransmitNanos) < periodicKeepAliveMs) {
- return;
- }
-
- LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.KEEPALIVE_RECORD_CONTENT);
- controlRec.setControl();
- asyncWrite(controlRec);
- }
-
-}
[21/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncFailureInjector.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncFailureInjector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncFailureInjector.java
deleted file mode 100644
index 4145040..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncFailureInjector.java
+++ /dev/null
@@ -1,131 +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.injector;
-
-/**
- * Failure Injector that works in asynchronous way
- */
-public interface AsyncFailureInjector {
-
- AsyncFailureInjector NULL = new AsyncFailureInjector() {
- @Override
- public void injectErrors(boolean enabled) {
- // no-op
- }
-
- @Override
- public boolean shouldInjectErrors() {
- return false;
- }
-
- @Override
- public void injectDelays(boolean enabled) {
- // no-op
- }
-
- @Override
- public boolean shouldInjectDelays() {
- return false;
- }
-
- @Override
- public int getInjectedDelayMs() {
- return 0;
- }
-
- @Override
- public void injectStops(boolean enabled) {
- // no-op
- }
-
- @Override
- public boolean shouldInjectStops() {
- return false;
- }
-
- @Override
- public boolean shouldInjectCorruption(long startEntryId, long endEntryId) {
- return false;
- }
-
- @Override
- public String toString() {
- return "NULL";
- }
- };
-
- /**
- * Enable or disable error injection.
- *
- * @param enabled
- * flag to enable or disable error injection.
- */
- void injectErrors(boolean enabled);
-
- /**
- * Return the flag indicating if should inject errors.
- *
- * @return true to inject errors otherwise false.
- */
- boolean shouldInjectErrors();
-
- /**
- * Enable or disable delay injection.
- *
- * @param enabled
- * flag to enable or disable delay injection.
- */
- void injectDelays(boolean enabled);
-
- /**
- * Return the flag indicating if should inject delays.
- *
- * @return true to inject delays otherwise false.
- */
- boolean shouldInjectDelays();
-
- /**
- * Return the injected delay in milliseconds.
- *
- * @return the injected delay in milliseconds.
- */
- int getInjectedDelayMs();
-
- /**
- * Enable or disable injecting stops. This could be used
- * for simulating stopping an action.
- */
- void injectStops(boolean enabled);
-
- /**
- * Return the flag indicating if should inject stops.
- *
- * @return true to inject stops otherwise false.
- */
- boolean shouldInjectStops();
-
- /**
- * Return the flag indicating if should inject corruption.
- *
- * @param startEntryId the start entry id
- * @param endEntryId the end entry id
- * @return true to inject corruption otherwise false.
- */
- boolean shouldInjectCorruption(long startEntryId, long endEntryId);
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncRandomFailureInjector.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncRandomFailureInjector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncRandomFailureInjector.java
deleted file mode 100644
index f3bfea9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncRandomFailureInjector.java
+++ /dev/null
@@ -1,186 +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.injector;
-
-import com.twitter.distributedlog.util.Utils;
-
-import java.util.Random;
-
-/**
- * Failure injector based on {@link java.util.Random}
- */
-public class AsyncRandomFailureInjector implements AsyncFailureInjector {
-
- private static final Random random = new Random(System.currentTimeMillis());
-
- public static Builder newBuilder() {
- return new Builder();
- }
-
- public static class Builder {
-
- private boolean _simulateDelays = false;
- private boolean _simulateErrors = false;
- private boolean _simulateStops = false;
- private boolean _simulateCorruption = false;
- private int _injectedDelayPercent = 0;
- private int _injectedErrorPercent = 0;
- private int _injectedStopPercent = 0;
- private int _maxInjectedDelayMs = Integer.MAX_VALUE;
-
- private Builder() {}
-
- public Builder injectDelays(boolean simulateDelays,
- int injectedDelayPercent,
- int maxInjectedDelayMs) {
- this._simulateDelays = simulateDelays;
- this._injectedDelayPercent = injectedDelayPercent;
- this._maxInjectedDelayMs = maxInjectedDelayMs;
- return this;
- }
-
- public Builder injectErrors(boolean simulateErrors,
- int injectedErrorPercent) {
- this._simulateErrors = simulateErrors;
- this._injectedErrorPercent = injectedErrorPercent;
- return this;
- }
-
- public Builder injectCorruption(boolean simulateCorruption) {
- this._simulateCorruption = simulateCorruption;
- return this;
- }
-
- public Builder injectStops(boolean simulateStops,
- int injectedStopPercent) {
- this._simulateStops = simulateStops;
- this._injectedStopPercent = injectedStopPercent;
- return this;
- }
-
- public AsyncFailureInjector build() {
- return new AsyncRandomFailureInjector(
- _simulateDelays,
- _injectedDelayPercent,
- _maxInjectedDelayMs,
- _simulateErrors,
- _injectedErrorPercent,
- _simulateStops,
- _injectedStopPercent,
- _simulateCorruption);
- }
-
- }
-
- private boolean simulateDelays;
- private boolean simulateErrors;
- private boolean simulateStops;
- private boolean simulateCorruption;
- private final int injectedDelayPercent;
- private final int injectedErrorPercent;
- private final int injectedStopPercent;
- private final int maxInjectedDelayMs;
-
- private AsyncRandomFailureInjector(boolean simulateDelays,
- int injectedDelayPercent,
- int maxInjectedDelayMs,
- boolean simulateErrors,
- int injectedErrorPercent,
- boolean simulateStops,
- int injectedStopPercent,
- boolean simulateCorruption) {
- this.simulateDelays = simulateDelays;
- this.injectedDelayPercent = injectedDelayPercent;
- this.maxInjectedDelayMs = maxInjectedDelayMs;
- this.simulateErrors = simulateErrors;
- this.injectedErrorPercent = injectedErrorPercent;
- this.simulateStops = simulateStops;
- this.injectedStopPercent = injectedStopPercent;
- this.simulateCorruption = simulateCorruption;
- }
-
- @Override
- public void injectErrors(boolean enabled) {
- this.simulateErrors = enabled;
- }
-
- @Override
- public boolean shouldInjectErrors() {
- return simulateErrors && Utils.randomPercent(injectedErrorPercent);
- }
-
- @Override
- public void injectDelays(boolean enabled) {
- this.simulateDelays = enabled;
- }
-
- @Override
- public boolean shouldInjectDelays() {
- return simulateDelays && Utils.randomPercent(injectedDelayPercent);
- }
-
- @Override
- public int getInjectedDelayMs() {
- if (maxInjectedDelayMs > 0) {
- return random.nextInt(maxInjectedDelayMs);
- }
- return 0;
- }
-
- @Override
- public void injectStops(boolean enabled) {
- this.simulateStops = enabled;
- }
-
- @Override
- public boolean shouldInjectStops() {
- return simulateStops && Utils.randomPercent(injectedStopPercent);
- }
-
- @Override
- public boolean shouldInjectCorruption(long startEntryId, long endEntryId) {
- if (!simulateCorruption) {
- return false;
- }
- if (startEntryId == endEntryId) {
- return startEntryId % 10 == 0;
- }
- for (long i = startEntryId; i <= endEntryId; i++) {
- if (i % 10 == 0) {
- return true;
- }
- }
- return false;
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("FailureInjector[");
- sb.append("errors=(").append(simulateErrors).append(", pct=")
- .append(injectedErrorPercent).append("), ");
- sb.append("delays=(").append(simulateDelays).append(", pct=")
- .append(injectedDelayPercent).append(", max=")
- .append(maxInjectedDelayMs).append("), ");
- sb.append("stops=(").append(simulateStops).append(", pct=")
- .append(injectedStopPercent).append(")");
- sb.append("corruption=(").append(simulateCorruption).append(")");
- sb.append("]");
- return sb.toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/FailureInjector.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/FailureInjector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/FailureInjector.java
deleted file mode 100644
index 16c8e4e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/FailureInjector.java
+++ /dev/null
@@ -1,37 +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.injector;
-
-/**
- * Failure injector.
- */
-public interface FailureInjector {
-
- /**
- * No-op failure injector, which does nothing.
- */
- public static FailureInjector NULL = new FailureInjector() {
- @Override
- public void inject() {
- // no-op;
- }
- };
-
- // inject failures
- void inject();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/RandomDelayFailureInjector.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/RandomDelayFailureInjector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/RandomDelayFailureInjector.java
deleted file mode 100644
index 73aad5b..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/RandomDelayFailureInjector.java
+++ /dev/null
@@ -1,60 +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.injector;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.util.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Injector that injects random delays
- */
-public class RandomDelayFailureInjector implements FailureInjector {
-
- private static final Logger LOG = LoggerFactory.getLogger(RandomDelayFailureInjector.class);
-
- private final DynamicDistributedLogConfiguration dynConf;
-
- public RandomDelayFailureInjector(DynamicDistributedLogConfiguration dynConf) {
- this.dynConf = dynConf;
- }
-
- private int delayMs() {
- return dynConf.getEIInjectedWriteDelayMs();
- }
-
- private double delayPct() {
- return dynConf.getEIInjectedWriteDelayPercent();
- }
-
- private boolean enabled() {
- return delayMs() > 0 && delayPct() > 0;
- }
-
- @Override
- public void inject() {
- try {
- if (enabled() && Utils.randomPercent(delayPct())) {
- Thread.sleep(delayMs());
- }
- } catch (InterruptedException ex) {
- LOG.warn("delay was interrupted ", ex);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/package-info.java
deleted file mode 100644
index ffee340..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Failure Injection
- */
-package com.twitter.distributedlog.injector;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortable.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortable.java
deleted file mode 100644
index 0d0b389..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortable.java
+++ /dev/null
@@ -1,41 +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.io;
-
-import java.io.IOException;
-
-/**
- * An {@code Abortable} is a source or destination of data that can be aborted.
- * The abort method is invoked to release resources that the object is holding
- * (such as open files). The abort happens when the object is in an error state,
- * which it couldn't be closed gracefully.
- *
- * @see java.io.Closeable
- * @since 0.3.32
- */
-public interface Abortable {
-
- /**
- * Aborts the object and releases any resources associated with it.
- * If the object is already aborted then invoking this method has no
- * effect.
- *
- * @throws IOException if an I/O error occurs.
- */
- public void abort() throws IOException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortables.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortables.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortables.java
deleted file mode 100644
index 4599574..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortables.java
+++ /dev/null
@@ -1,183 +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.io;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.function.VoidFunctions;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-
-/**
- * Utility methods for working with {@link Abortable} objects.
- *
- * @since 0.3.32
- */
-public final class Abortables {
-
- static final Logger logger = LoggerFactory.getLogger(Abortables.class);
-
- private Abortables() {}
-
- public static Future<Void> asyncAbort(@Nullable AsyncAbortable abortable,
- boolean swallowIOException) {
- if (null == abortable) {
- return Future.Void();
- } else if (swallowIOException) {
- return FutureUtils.ignore(abortable.asyncAbort());
- } else {
- return abortable.asyncAbort();
- }
- }
-
- /**
- * Aborts a {@link Abortable}, with control over whether an {@link IOException} may be thrown.
- * This is primarily useful in a finally block, where a thrown exception needs to be logged but
- * not propagated (otherwise the original exception will be lost).
- *
- * <p>If {@code swallowIOException} is true then we never throw {@code IOException} but merely log it.
- *
- * <p>Example: <pre> {@code
- *
- * public void abortStreamNicely() throws IOException {
- * SomeStream stream = new SomeStream("foo");
- * try {
- * // ... code which does something with the stream ...
- * } catch (IOException ioe) {
- * // If an exception occurs, we might abort the stream.
- * Abortables.abort(stream, true);
- * }
- * }}</pre>
- *
- * @param abortable the {@code Abortable} object to be aborted, or null, in which case this method
- * does nothing.
- * @param swallowIOException if true, don't propagate IO exceptions thrown by the {@code abort} methods
- * @throws IOException if {@code swallowIOException} is false and {@code abort} throws an {@code IOException}
- */
- public static void abort(@Nullable Abortable abortable,
- boolean swallowIOException)
- throws IOException {
- if (null == abortable) {
- return;
- }
- try {
- abortable.abort();
- } catch (IOException ioe) {
- if (swallowIOException) {
- logger.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
- } else {
- throw ioe;
- }
- }
- }
-
- /**
- * Abort async <i>abortable</i>
- *
- * @param abortable the {@code AsyncAbortable} object to be aborted, or null, in which case this method
- * does nothing.
- * @param swallowIOException if true, don't propagate IO exceptions thrown by the {@code abort} methods
- * @throws IOException if {@code swallowIOException} is false and {@code abort} throws an {@code IOException}
- * @see #abort(Abortable, boolean)
- */
- public static void abort(@Nullable AsyncAbortable abortable,
- boolean swallowIOException)
- throws IOException {
- if (null == abortable) {
- return;
- }
- try {
- FutureUtils.result(abortable.asyncAbort());
- } catch (IOException ioe) {
- if (swallowIOException) {
- logger.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
- } else {
- throw ioe;
- }
- }
- }
-
- /**
- * Aborts the given {@code abortable}, logging any {@code IOException} that's thrown rather than
- * propagating it.
- *
- * While it's not safe in the general case to ignore exceptions that are thrown when aborting an
- * I/O resource, it should generally be safe in the case of a resource that's being used only for
- * reading.
- *
- * @param abortable the {@code Abortable} to be closed, or {@code null} in which case this method
- * does nothing.
- */
- public static void abortQuietly(@Nullable Abortable abortable) {
- try {
- abort(abortable, true);
- } catch (IOException e) {
- logger.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
- }
- }
-
- /**
- * Aborts the given {@code abortable}, logging any {@code IOException} that's thrown rather than
- * propagating it.
- *
- * While it's not safe in the general case to ignore exceptions that are thrown when aborting an
- * I/O resource, it should generally be safe in the case of a resource that's being used only for
- * reading.
- *
- * @param abortable the {@code AsyncAbortable} to be closed, or {@code null} in which case this method
- * does nothing.
- */
- public static void abortQuietly(@Nullable AsyncAbortable abortable) {
- try {
- abort(abortable, true);
- } catch (IOException e) {
- logger.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
- }
- }
-
- /**
- * Abort the abortables in sequence.
- *
- * @param executorService
- * executor service to execute
- * @param abortables
- * abortables to abort
- * @return future represents the abort future
- */
- public static Future<Void> abortSequence(ExecutorService executorService,
- AsyncAbortable... abortables) {
- List<AsyncAbortable> abortableList = Lists.newArrayListWithExpectedSize(abortables.length);
- for (AsyncAbortable abortable : abortables) {
- if (null == abortable) {
- abortableList.add(AsyncAbortable.NULL);
- } else {
- abortableList.add(abortable);
- }
- }
- return FutureUtils.processList(
- abortableList,
- AsyncAbortable.ABORT_FUNC,
- executorService).map(VoidFunctions.LIST_TO_VOID_FUNC);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncAbortable.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncAbortable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncAbortable.java
deleted file mode 100644
index ed1062a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncAbortable.java
+++ /dev/null
@@ -1,57 +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.io;
-
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-
-/**
- * An {@code Abortable} is a source or destination of data that can be aborted.
- * The abort method is invoked to release resources that the object is holding
- * (such as open files). The abort happens when the object is in an error state,
- * which it couldn't be closed gracefully.
- *
- * @see AsyncCloseable
- * @see Abortable
- * @since 0.3.43
- */
-public interface AsyncAbortable {
-
- Function<AsyncAbortable, Future<Void>> ABORT_FUNC = new Function<AsyncAbortable, Future<Void>>() {
- @Override
- public Future<Void> apply(AsyncAbortable abortable) {
- return abortable.asyncAbort();
- }
- };
-
- AsyncAbortable NULL = new AsyncAbortable() {
- @Override
- public Future<Void> asyncAbort() {
- return Future.Void();
- }
- };
-
- /**
- * Aborts the object and releases any resources associated with it.
- * If the object is already aborted then invoking this method has no
- * effect.
- *
- * @return future represents the abort result
- */
- Future<Void> asyncAbort();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncCloseable.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncCloseable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncCloseable.java
deleted file mode 100644
index 817a8e2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncCloseable.java
+++ /dev/null
@@ -1,60 +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.io;
-
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-
-/**
- * A {@code AsyncCloseable} is a source or destination of data that can be closed asynchronously.
- * The close method is invoked to release resources that the object is
- * holding (such as open files).
- */
-public interface AsyncCloseable {
-
- Function<AsyncCloseable, Future<Void>> CLOSE_FUNC = new Function<AsyncCloseable, Future<Void>>() {
- @Override
- public Future<Void> apply(AsyncCloseable closeable) {
- return closeable.asyncClose();
- }
- };
-
- Function<AsyncCloseable, Future<Void>> CLOSE_FUNC_IGNORE_ERRORS = new Function<AsyncCloseable, Future<Void>>() {
- @Override
- public Future<Void> apply(AsyncCloseable closeable) {
- return FutureUtils.ignore(closeable.asyncClose());
- }
- };
-
- AsyncCloseable NULL = new AsyncCloseable() {
- @Override
- public Future<Void> asyncClose() {
- return Future.Void();
- }
- };
-
- /**
- * Closes this source and releases any system resources associated
- * with it. If the source is already closed then invoking this
- * method has no effect.
- *
- * @return future representing the close result.
- */
- Future<Void> asyncClose();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncDeleteable.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncDeleteable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncDeleteable.java
deleted file mode 100644
index 203895e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncDeleteable.java
+++ /dev/null
@@ -1,34 +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.io;
-
-import com.twitter.util.Future;
-
-/**
- * A {@code AsyncDeleteable} is a source or destination of data that can be deleted asynchronously.
- * This delete method is invoked to delete the source.
- */
-public interface AsyncDeleteable {
- /**
- * Releases any system resources associated with this and delete the source. If the source is
- * already deleted then invoking this method has no effect.
- *
- * @return future representing the deletion result.
- */
- Future<Void> delete();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/package-info.java
deleted file mode 100644
index df2e91f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * IO Utils for distributedlog
- */
-package com.twitter.distributedlog.io;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ChainedRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ChainedRequestLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ChainedRequestLimiter.java
deleted file mode 100644
index 60eacd5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ChainedRequestLimiter.java
+++ /dev/null
@@ -1,76 +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.limiter;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.ImmutableList;
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Chain request limiters for easier management of multi limiter policy.
- */
-public class ChainedRequestLimiter<Request> implements RequestLimiter<Request> {
- private final ImmutableList<RequestLimiter<Request>> limiters;
- private final OpStatsLogger applyTime;
-
- public static class Builder<Request> {
- private final ImmutableList.Builder<RequestLimiter<Request>> limitersBuilder;
- private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
-
- public Builder() {
- this.limitersBuilder = new ImmutableList.Builder<RequestLimiter<Request>>();
- }
-
- public Builder<Request> addLimiter(RequestLimiter<Request> limiter) {
- this.limitersBuilder.add(limiter);
- return this;
- }
-
- public Builder<Request> statsLogger(StatsLogger statsLogger) {
- this.statsLogger = statsLogger;
- return this;
- }
-
- public ChainedRequestLimiter<Request> build() {
- return new ChainedRequestLimiter<Request>(limitersBuilder.build(), statsLogger);
- }
- }
-
- private ChainedRequestLimiter(ImmutableList<RequestLimiter<Request>> limiters,
- StatsLogger statsLogger) {
- this.limiters = limiters;
- this.applyTime = statsLogger.getOpStatsLogger("apply");
- }
-
- public void apply(Request request) throws OverCapacityException {
- Stopwatch stopwatch = Stopwatch.createStarted();
- try {
- for (RequestLimiter<Request> limiter : limiters) {
- limiter.apply(request);
- }
- } finally {
- applyTime.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ComposableRequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ComposableRequestLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ComposableRequestLimiter.java
deleted file mode 100644
index 55e4c8b..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ComposableRequestLimiter.java
+++ /dev/null
@@ -1,73 +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.limiter;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.limiter.GuavaRateLimiter;
-import com.twitter.distributedlog.limiter.RateLimiter;
-
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Collect rate limiter implementation, cost(Request), overlimit, etc. behavior.
- */
-public class ComposableRequestLimiter<Request> implements RequestLimiter<Request> {
- protected static final Logger LOG = LoggerFactory.getLogger(ComposableRequestLimiter.class);
-
- private final RateLimiter limiter;
- private final OverlimitFunction<Request> overlimitFunction;
- private final CostFunction<Request> costFunction;
- private final Counter overlimitCounter;
-
- static public interface OverlimitFunction<Request> {
- void apply(Request request) throws OverCapacityException;
- }
- static public interface CostFunction<Request> {
- int apply(Request request);
- }
-
- public ComposableRequestLimiter(
- RateLimiter limiter,
- OverlimitFunction<Request> overlimitFunction,
- CostFunction<Request> costFunction,
- StatsLogger statsLogger) {
- Preconditions.checkNotNull(limiter);
- Preconditions.checkNotNull(overlimitFunction);
- Preconditions.checkNotNull(costFunction);
- this.limiter = limiter;
- this.overlimitFunction = overlimitFunction;
- this.costFunction = costFunction;
- this.overlimitCounter = statsLogger.getCounter("overlimit");
- }
-
- @Override
- public void apply(Request request) throws OverCapacityException {
- int permits = costFunction.apply(request);
- if (!limiter.acquire(permits)) {
- overlimitCounter.inc();
- overlimitFunction.apply(request);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/GuavaRateLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/GuavaRateLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/GuavaRateLimiter.java
deleted file mode 100644
index 3f1909a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/GuavaRateLimiter.java
+++ /dev/null
@@ -1,58 +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.limiter;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Wrap a guava limiter in a simple interface to make testing easier.
- * Notes:
- * 1. Negative limit translates into (virtually) unlimited.
- * 2. Calling acquire with permits == 0 translates into no acquire.
- */
-public class GuavaRateLimiter implements RateLimiter {
- com.google.common.util.concurrent.RateLimiter limiter;
-
- public static RateLimiter of(int limit) {
- if (limit == 0) {
- return RateLimiter.REJECT;
- } else if (limit < 0) {
- return RateLimiter.ACCEPT;
- } else {
- return new GuavaRateLimiter(limit);
- }
- }
-
- public GuavaRateLimiter(int limit) {
- double effectiveLimit = limit;
- if (limit < 0) {
- effectiveLimit = Double.POSITIVE_INFINITY;
- }
- this.limiter = com.google.common.util.concurrent.RateLimiter.create(effectiveLimit);
- }
-
- @Override
- public boolean acquire(int permits) {
- Preconditions.checkState(permits >= 0);
- if (permits > 0) {
- return limiter.tryAcquire(permits);
- } else {
- return true;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RateLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RateLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RateLimiter.java
deleted file mode 100644
index 0cb1ebe..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RateLimiter.java
+++ /dev/null
@@ -1,49 +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.limiter;
-
-/**
- * Simple interface for a rate limiter used by RequestLimiter.
- */
-public interface RateLimiter {
-
- public static final RateLimiter REJECT = new RateLimiter() {
- @Override
- public boolean acquire(int permits) {
- return false;
- }
- };
-
- public static final RateLimiter ACCEPT = new RateLimiter() {
- @Override
- public boolean acquire(int permits) {
- return true;
- }
- };
-
- public static abstract class Builder {
- public abstract RateLimiter build();
- }
-
- /**
- * Try to acquire a certain number of permits.
- *
- * @param permits number of permits to acquire
- */
- boolean acquire(int permits);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RequestLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RequestLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RequestLimiter.java
deleted file mode 100644
index 6c5ad96..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RequestLimiter.java
+++ /dev/null
@@ -1,24 +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.limiter;
-
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-
-public interface RequestLimiter<Request> {
- public void apply(Request request) throws OverCapacityException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/package-info.java
deleted file mode 100644
index d5f61a8..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Rate limiting for distributedlog
- */
-package com.twitter.distributedlog.limiter;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLock.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLock.java
deleted file mode 100644
index fa8bdf0..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLock.java
+++ /dev/null
@@ -1,54 +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.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-/**
- * Interface for distributed locking
- */
-public interface DistributedLock extends AsyncCloseable {
-
- /**
- * Asynchronously acquire the lock.
- *
- * @return future represents the acquire result.
- */
- Future<? extends DistributedLock> asyncAcquire();
-
- /**
- * Check if hold lock. If it doesn't, then re-acquire the lock.
- *
- * @throws LockingException if the lock attempt fails
- * @see #checkOwnership()
- */
- void checkOwnershipAndReacquire() throws LockingException;
-
- /**
- * Check if the lock is held. If not, error out and do not re-acquire.
- * Use this in cases where there are many waiters by default and re-acquire
- * is unlikely to succeed.
- *
- * @throws LockingException if we lost the ownership
- * @see #checkOwnershipAndReacquire()
- */
- void checkOwnership() throws LockingException;
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLockContext.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLockContext.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLockContext.java
deleted file mode 100644
index 1914793..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLockContext.java
+++ /dev/null
@@ -1,43 +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.lock;
-
-import org.apache.commons.lang3.tuple.Pair;
-
-import java.util.HashSet;
-import java.util.Set;
-
-class DistributedLockContext {
- private final Set<Pair<String, Long>> lockIds;
-
- DistributedLockContext() {
- this.lockIds = new HashSet<Pair<String, Long>>();
- }
-
- synchronized void addLockId(Pair<String, Long> lockId) {
- this.lockIds.add(lockId);
- }
-
- synchronized void clearLockIds() {
- this.lockIds.clear();
- }
-
- synchronized boolean hasLockId(Pair<String, Long> lockId) {
- return this.lockIds.contains(lockId);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/EpochChangedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/EpochChangedException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/EpochChangedException.java
deleted file mode 100644
index 032a9cd..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/EpochChangedException.java
+++ /dev/null
@@ -1,33 +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.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-
-/**
- * Exception indicates that epoch already changed when executing a given
- * {@link LockAction}.
- */
-public class EpochChangedException extends LockingException {
-
- private static final long serialVersionUID = 8775257025963870331L;
-
- public EpochChangedException(String lockPath, int expectedEpoch, int currentEpoch) {
- super(lockPath, "lock " + lockPath + " already moved to epoch " + currentEpoch + ", expected " + expectedEpoch);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockAction.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockAction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockAction.java
deleted file mode 100644
index 46b420d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockAction.java
+++ /dev/null
@@ -1,36 +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.lock;
-
-/**
- * Lock Action
- */
-interface LockAction {
-
- /**
- * Execute a lock action
- */
- void execute();
-
- /**
- * Get lock action name.
- *
- * @return lock action name
- */
- String getActionName();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockClosedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockClosedException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockClosedException.java
deleted file mode 100644
index 5b676bf..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockClosedException.java
+++ /dev/null
@@ -1,38 +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.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.lock.ZKSessionLock.State;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * Exception indicates that the lock was closed (unlocked) before the lock request could complete.
- */
-public class LockClosedException extends LockingException {
-
- private static final long serialVersionUID = 8775257025963470331L;
-
- public LockClosedException(String lockPath, String msg) {
- super(lockPath, msg);
- }
-
- public LockClosedException(String lockPath, Pair<String, Long> lockId, State currentState) {
- super(lockPath, "lock at path " + lockPath + " with id " + lockId + " closed early in state : " + currentState);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockListener.java
deleted file mode 100644
index 681c180..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockListener.java
+++ /dev/null
@@ -1,28 +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.lock;
-
-/**
- * Listener on lock state changes
- */
-interface LockListener {
- /**
- * Triggered when a lock is changed from CLAIMED to EXPIRED.
- */
- void onExpired();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockSessionExpiredException.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockSessionExpiredException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockSessionExpiredException.java
deleted file mode 100644
index dac1253..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockSessionExpiredException.java
+++ /dev/null
@@ -1,34 +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.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.lock.ZKSessionLock.State;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * Exception indicates that the lock's zookeeper session was expired before the lock request could complete.
- */
-public class LockSessionExpiredException extends LockingException {
-
- private static final long serialVersionUID = 8775253025963470331L;
-
- public LockSessionExpiredException(String lockPath, Pair<String, Long> lockId, State currentState) {
- super(lockPath, "lock at path " + lockPath + " with id " + lockId + " expired early in state : " + currentState);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockStateChangedException.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockStateChangedException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockStateChangedException.java
deleted file mode 100644
index 2b99795..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockStateChangedException.java
+++ /dev/null
@@ -1,36 +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.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.lock.ZKSessionLock.State;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * Exception thrown when lock state changed
- */
-public class LockStateChangedException extends LockingException {
-
- private static final long serialVersionUID = -3770866789942102262L;
-
- LockStateChangedException(String lockPath, Pair<String, Long> lockId,
- State expectedState, State currentState) {
- super(lockPath, "Lock state of " + lockId + " for " + lockPath + " has changed : expected "
- + expectedState + ", but " + currentState);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockTimeoutException.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockTimeoutException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockTimeoutException.java
deleted file mode 100644
index 3020980..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockTimeoutException.java
+++ /dev/null
@@ -1,34 +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.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Exception thrown when acquiring lock timeout
- */
-public class LockTimeoutException extends LockingException {
-
- private static final long serialVersionUID = -3837638877423323820L;
-
- LockTimeoutException(String lockPath, long timeout, TimeUnit unit) {
- super(lockPath, "Locking " + lockPath + " timeout in " + timeout + " " + unit);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockWaiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockWaiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockWaiter.java
deleted file mode 100644
index 73ffabc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockWaiter.java
+++ /dev/null
@@ -1,95 +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.lock;
-
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import com.twitter.util.Timer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Lock waiter represents the attempt that application tries to lock.
- */
-public class LockWaiter {
-
- private static final Logger logger = LoggerFactory.getLogger(LockWaiter.class);
-
- private final String lockId;
- private final String currentOwner;
- private final Future<Boolean> acquireFuture;
-
- public LockWaiter(String lockId,
- String currentOwner,
- Future<Boolean> acquireFuture) {
- this.lockId = lockId;
- this.currentOwner = currentOwner;
- this.acquireFuture = acquireFuture;
- }
-
- /**
- * Return the lock id of the waiter.
- *
- * @return lock id of the waiter
- */
- public String getId() {
- return lockId;
- }
-
- /**
- * Return the owner that observed when waiter is waiting.
- *
- * @return the owner that observed when waiter is waiting
- */
- public String getCurrentOwner() {
- return currentOwner;
- }
-
- /**
- * Return the future representing the waiting result.
- *
- * <p>If the future is interrupted (e.g. {@link Future#within(Duration, Timer)}),
- * the waiter will automatically clean up its waiting state.
- *
- * @return the future representing the acquire result.
- */
- public Future<Boolean> getAcquireFuture() {
- return acquireFuture;
- }
-
- /**
- * Wait for the acquire result.
- *
- * @return true if acquired successfully, otherwise false.
- */
- public boolean waitForAcquireQuietly() {
- boolean success = false;
- try {
- success = Await.result(acquireFuture);
- } catch (InterruptedException ie) {
- Thread.currentThread().interrupt();
- } catch (LockTimeoutException lte) {
- logger.debug("Timeout on lock acquiring", lte);
- } catch (Exception e) {
- logger.error("Caught exception waiting for lock acquired", e);
- }
- return success;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/NopDistributedLock.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/NopDistributedLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/NopDistributedLock.java
deleted file mode 100644
index ef6b9ab..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/NopDistributedLock.java
+++ /dev/null
@@ -1,51 +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.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.util.Future;
-
-/**
- * An implementation of {@link DistributedLock} which does nothing.
- */
-public class NopDistributedLock implements DistributedLock {
-
- public static final DistributedLock INSTANCE = new NopDistributedLock();
-
- private NopDistributedLock() {}
-
- @Override
- public Future<? extends DistributedLock> asyncAcquire() {
- return Future.value(this);
- }
-
- @Override
- public void checkOwnershipAndReacquire() throws LockingException {
- // no-op
- }
-
- @Override
- public void checkOwnership() throws LockingException {
- // no-op
- }
-
- @Override
- public Future<Void> asyncClose() {
- return Future.Void();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLock.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLock.java
deleted file mode 100644
index 95cd593..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLock.java
+++ /dev/null
@@ -1,126 +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.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.util.Future;
-import scala.runtime.BoxedUnit;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * One time lock.
- * <p>The lock is only alive during a given period. It should
- * be not usable if the lock is expired.
- * <p>Listener could be registered by {@link #setLockListener(LockListener)}
- * to receive state changes of the lock.
- */
-public interface SessionLock {
-
- /**
- * Set lock listener for lock state changes.
- * <p>Typically a listener should be set before try locking.
- *
- * @param lockListener
- * lock listener for state changes.
- */
- SessionLock setLockListener(LockListener lockListener);
-
- /**
- * Whether the lock is held or not?
- *
- * @return true if the lock is held, otherwise false.
- */
- boolean isLockHeld();
-
- /**
- * Whether the lock is expired or not?
- * <p>If a lock is expired, it will not be reusable any more. Because it is an one-time lock.
- *
- * @return true if the lock is expired, otherwise false.
- */
- boolean isLockExpired();
-
- /**
- * Acquire the lock if it is free within given waiting time.
- * <p>
- * Calling this method will attempt to acquire the lock. If the lock
- * is already acquired by others, the caller will wait for <i>timeout</i>
- * period. If the caller could claim the lock within <i>timeout</i> period,
- * the caller acquire the lock. Otherwise, it would fail with {@link OwnershipAcquireFailedException}.
- * <p>
- * {@link #unlock()} should be called to unlock a claimed lock. The caller
- * doesn't need to unlock to clean up resources if <i>tryLock</i> fails.
- * <p>
- * <i>tryLock</i> here is effectively the combination of following asynchronous calls.
- * <pre>
- * ZKDistributedLock lock = ...;
- * Future<LockWaiter> attemptFuture = lock.asyncTryLock(...);
- *
- * boolean acquired = waiter.waitForAcquireQuietly();
- * if (acquired) {
- * // ...
- * } else {
- * // ...
- * }
- * </pre>
- *
- * @param timeout
- * timeout period to wait for claiming ownership
- * @param unit
- * unit of timeout period
- * @throws OwnershipAcquireFailedException if the lock is already acquired by others
- * @throws LockingException when encountered other lock related issues.
- */
- void tryLock(long timeout, TimeUnit unit)
- throws OwnershipAcquireFailedException, LockingException;
-
- /**
- * Acquire the lock in asynchronous way.
- * <p>
- * Calling this method will attempt to place a lock waiter to acquire this lock.
- * The future returned by this method represents the result of this attempt. It doesn't mean
- * the caller acquired the lock or not. The application should check {@link LockWaiter#getAcquireFuture()}
- * to see if it acquired the lock or not.
- *
- * @param timeout
- * timeout period to wait for claiming ownership
- * @param unit
- * unit of timeout period
- * @return lock waiter representing this attempt of acquiring lock.
- * @see #tryLock(long, TimeUnit)
- */
- Future<LockWaiter> asyncTryLock(long timeout, TimeUnit unit);
-
- /**
- * Release a claimed lock.
- *
- * @see #tryLock(long, TimeUnit)
- */
- void unlock();
-
- /**
- * Release a claimed lock in the asynchronous way.
- *
- * @return future representing the result of unlock operation.
- * @see #unlock()
- */
- Future<BoxedUnit> asyncUnlock();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLockFactory.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLockFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLockFactory.java
deleted file mode 100644
index 4334626..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLockFactory.java
+++ /dev/null
@@ -1,38 +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.lock;
-
-import com.twitter.util.Future;
-
-/**
- * Factory to create {@link SessionLock}
- */
-public interface SessionLockFactory {
-
- /**
- * Create a lock with lock path.
- *
- * @param lockPath
- * lock path
- * @param context
- * lock context
- * @return future represents the creation result.
- */
- Future<SessionLock> createLock(String lockPath, DistributedLockContext context);
-
-}
[31/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
deleted file mode 100644
index 6da4b8d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
+++ /dev/null
@@ -1,3528 +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.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.bk.QuorumConfig;
-import com.twitter.distributedlog.feature.DefaultFeatureProvider;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.net.DNSResolverForRacks;
-import com.twitter.distributedlog.net.DNSResolverForRows;
-import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.configuration.CompositeConfiguration;
-import org.apache.commons.configuration.Configuration;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.PropertiesConfiguration;
-import org.apache.commons.configuration.SystemConfiguration;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Set;
-
-/**
- * DistributedLog Configuration.
- * <p>
- * DistributedLog configuration is basically a properties based configuration, which extends from
- * Apache commons {@link CompositeConfiguration}. All the DL settings are in camel case and prefixed
- * with a meaningful component name. for example, `zkSessionTimeoutSeconds` means <i>SessionTimeoutSeconds</i>
- * for component `zk`.
- *
- * <h3>BookKeeper Configuration</h3>
- *
- * BookKeeper client configuration settings could be loaded via DistributedLog configuration. All those
- * settings are prefixed with <i>`bkc.`</i>. For example, <i>bkc.zkTimeout</i> in distributedlog configuration
- * will be applied as <i>`zkTimeout`</i> in bookkeeper client configuration.
- *
- * <h3>How to load configuration</h3>
- *
- * The default distributedlog configuration is constructed by instantiated a new instance. This
- * distributedlog configuration will automatically load the settings that specified via
- * {@link SystemConfiguration}.
- *
- * <pre>
- * DistributedLogConfiguration conf = new DistributedLogConfiguration();
- * </pre>
- *
- * The recommended way is to load configuration from URL that points to a configuration file
- * ({@link #loadConf(URL)}).
- *
- * <pre>
- * String configFile = "/path/to/distributedlog/conf/file";
- * DistributedLogConfiguration conf = new DistributedLogConfiguration();
- * conf.loadConf(new File(configFile).toURI().toURL());
- * </pre>
- *
- * @see org.apache.bookkeeper.conf.ClientConfiguration
- */
-public class DistributedLogConfiguration extends CompositeConfiguration {
- static final Logger LOG = LoggerFactory.getLogger(DistributedLogConfiguration.class);
-
- private static ClassLoader defaultLoader;
-
- static {
- defaultLoader = Thread.currentThread().getContextClassLoader();
- if (null == defaultLoader) {
- defaultLoader = DistributedLogConfiguration.class.getClassLoader();
- }
- }
-
- //
- // ZooKeeper Related Settings
- //
-
- public static final String BKDL_ZK_ACL_ID = "zkAclId";
- public static final String BKDL_ZK_ACL_ID_DEFAULT = null;
- public static final String BKDL_ZK_SESSION_TIMEOUT_SECONDS = "zkSessionTimeoutSeconds";
- public static final int BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT = 30;
- public static final String BKDL_ZK_REQUEST_RATE_LIMIT = "zkRequestRateLimit";
- public static final double BKDL_ZK_REQUEST_RATE_LIMIT_DEFAULT = 0;
- public static final String BKDL_ZK_NUM_RETRIES = "zkNumRetries";
- public static final int BKDL_ZK_NUM_RETRIES_DEFAULT = 3;
- public static final String BKDL_ZK_RETRY_BACKOFF_START_MILLIS = "zkRetryStartBackoffMillis";
- public static final int BKDL_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT = 5000;
- public static final String BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS = "zkRetryMaxBackoffMillis";
- public static final int BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT = 30000;
- public static final String BKDL_ZKCLIENT_NUM_RETRY_THREADS = "zkcNumRetryThreads";
- public static final int BKDL_ZKCLIENT_NUM_RETRY_THREADS_DEFAULT = 1;
-
- //
- // BookKeeper Related Settings
- //
-
- // BookKeeper zookeeper settings
- public static final String BKDL_BKCLIENT_ZK_SESSION_TIMEOUT = "bkcZKSessionTimeoutSeconds";
- public static final int BKDL_BKCLIENT_ZK_SESSION_TIMEOUT_DEFAULT = 30;
- public static final String BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT = "bkcZKRequestRateLimit";
- public static final double BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT_DEFAULT = 0;
- public static final String BKDL_BKCLIENT_ZK_NUM_RETRIES = "bkcZKNumRetries";
- public static final int BKDL_BKCLIENT_ZK_NUM_RETRIES_DEFAULT = 3;
- public static final String BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS = "bkcZKRetryStartBackoffMillis";
- public static final int BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT = 5000;
- public static final String BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS = "bkcZKRetryMaxBackoffMillis";
- public static final int BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT = 30000;
-
- // Bookkeeper ensemble placement settings
- // Bookkeeper ensemble size
- public static final String BKDL_BOOKKEEPER_ENSEMBLE_SIZE = "bkcEnsembleSize";
- // @Deprecated
- public static final String BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD = "ensemble-size";
- public static final int BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT = 3;
- // Bookkeeper write quorum size
- public static final String BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE = "bkcWriteQuorumSize";
- // @Deprecated
- public static final String BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD = "write-quorum-size";
- public static final int BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT = 3;
- // Bookkeeper ack quorum size
- public static final String BKDL_BOOKKEEPER_ACK_QUORUM_SIZE = "bkcAckQuorumSize";
- // @Deprecated
- public static final String BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD = "ack-quorum-size";
- public static final int BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT = 2;
- public static final String BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT = "bkRowAwareEnsemblePlacement";
- public static final String BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_OLD = "row-aware-ensemble-placement";
- public static final boolean BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_DEFAULT = false;
- public static final String BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS = "bkEnsemblePlacementDnsResolverClass";
- public static final String BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS_DEFAULT =
- DNSResolverForRacks.class.getName();
- public static final String BKDL_BK_DNS_RESOLVER_OVERRIDES = "dnsResolverOverrides";
- public static final String BKDL_BK_DNS_RESOLVER_OVERRIDES_DEFAULT = "";
-
- // General Settings
- // @Deprecated
- public static final String BKDL_BOOKKEEPER_DIGEST_PW = "digestPw";
- public static final String BKDL_BOOKKEEPER_DIGEST_PW_DEFAULT = "";
- public static final String BKDL_BKCLIENT_NUM_IO_THREADS = "bkcNumIOThreads";
- public static final String BKDL_TIMEOUT_TIMER_TICK_DURATION_MS = "timerTickDuration";
- public static final long BKDL_TIMEOUT_TIMER_TICK_DURATION_MS_DEFAULT = 100;
- public static final String BKDL_TIMEOUT_TIMER_NUM_TICKS = "timerNumTicks";
- public static final int BKDL_TIMEOUT_TIMER_NUM_TICKS_DEFAULT = 1024;
-
- //
- // Deprecated BookKeeper Settings (in favor of "bkc." style bookkeeper settings)
- //
-
- public static final String BKDL_BKCLIENT_READ_TIMEOUT = "bkcReadTimeoutSeconds";
- public static final int BKDL_BKCLIENT_READ_TIMEOUT_DEFAULT = 10;
- public static final String BKDL_BKCLIENT_WRITE_TIMEOUT = "bkcWriteTimeoutSeconds";
- public static final int BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT = 10;
- public static final String BKDL_BKCLIENT_NUM_WORKER_THREADS = "bkcNumWorkerThreads";
- public static final int BKDL_BKCLEINT_NUM_WORKER_THREADS_DEFAULT = 1;
-
- //
- // DL General Settings
- //
-
- // Executor Parameters
- public static final String BKDL_NUM_WORKER_THREADS = "numWorkerThreads";
- public static final String BKDL_NUM_READAHEAD_WORKER_THREADS = "numReadAheadWorkerThreads";
- public static final String BKDL_NUM_LOCKSTATE_THREADS = "numLockStateThreads";
- public static final String BKDL_NUM_RESOURCE_RELEASE_THREADS = "numResourceReleaseThreads";
- public static final String BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS = "schedulerShutdownTimeoutMs";
- public static final int BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS_DEFAULT = 5000;
- public static final String BKDL_USE_DAEMON_THREAD = "useDaemonThread";
- public static final boolean BKDL_USE_DAEMON_THREAD_DEFAULT = false;
-
- // Metadata Parameters
- public static final String BKDL_LEDGER_METADATA_LAYOUT_VERSION = "ledgerMetadataLayoutVersion";
- public static final String BKDL_LEDGER_METADATA_LAYOUT_VERSION_OLD = "ledger-metadata-layout";
- public static final int BKDL_LEDGER_METADATA_LAYOUT_VERSION_DEFAULT =
- LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
- public static final String BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK = "ledgerMetadataSkipMinVersionCheck";
- public static final boolean BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK_DEFAULT = false;
- public static final String BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER = "firstLogsegmentSequenceNumber";
- public static final String BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_OLD = "first-logsegment-sequence-number";
- public static final long BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_DEFAULT =
- DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO;
- public static final String BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED = "logSegmentSequenceNumberValidationEnabled";
- public static final boolean BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED_DEFAULT = true;
- public static final String BKDL_ENABLE_RECORD_COUNTS = "enableRecordCounts";
- public static final boolean BKDL_ENABLE_RECORD_COUNTS_DEFAULT = true;
- public static final String BKDL_MAXID_SANITYCHECK = "maxIdSanityCheck";
- public static final boolean BKDL_MAXID_SANITYCHECK_DEFAULT = true;
- public static final String BKDL_ENCODE_REGION_ID_IN_VERSION = "encodeRegionIDInVersion";
- public static final boolean BKDL_ENCODE_REGION_ID_IN_VERSION_DEFAULT = false;
- // (@Deprecated)
- public static final String BKDL_LOGSEGMENT_NAME_VERSION = "logSegmentNameVersion";
- public static final int BKDL_LOGSEGMENT_NAME_VERSION_DEFAULT = DistributedLogConstants.LOGSEGMENT_NAME_VERSION;
- // (@Derepcated) Name for the default (non-partitioned) stream
- public static final String BKDL_UNPARTITIONED_STREAM_NAME = "unpartitionedStreamName";
- public static final String BKDL_UNPARTITIONED_STREAM_NAME_DEFAULT = "<default>";
-
- // Log Segment Cache Parameters
- public static final String BKDL_LOGSEGMENT_CACHE_TTL_MS = "logSegmentCacheTTLMs";
- public static final long BKDL_LOGSEGMENT_CACHE_TTL_MS_DEFAULT = 600000; // 10 mins
- public static final String BKDL_LOGSEGMENT_CACHE_MAX_SIZE = "logSegmentCacheMaxSize";
- public static final long BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT = 10000;
- public static final String BKDL_LOGSEGMENT_CACHE_ENABLED = "logSegmentCacheEnabled";
- public static final boolean BKDL_LOGSEGMENT_CACHE_ENABLED_DEFAULT = true;
-
- //
- // DL Writer Settings
- //
-
- // General Settings
- public static final String BKDL_CREATE_STREAM_IF_NOT_EXISTS = "createStreamIfNotExists";
- public static final boolean BKDL_CREATE_STREAM_IF_NOT_EXISTS_DEFAULT = true;
- public static final String BKDL_LOG_FLUSH_TIMEOUT = "logFlushTimeoutSeconds";
- public static final int BKDL_LOG_FLUSH_TIMEOUT_DEFAULT = 30;
- /**
- * CompressionCodec.Type String to use (See CompressionUtils)
- * --------------------- ------------------------------------
- * NONE none
- * LZ4 lz4
- * UNKNOWN any other instance of String.class
- */
- public static final String BKDL_COMPRESSION_TYPE = "compressionType";
- public static final String BKDL_COMPRESSION_TYPE_DEFAULT = "none";
- public static final String BKDL_FAILFAST_ON_STREAM_NOT_READY = "failFastOnStreamNotReady";
- public static final boolean BKDL_FAILFAST_ON_STREAM_NOT_READY_DEFAULT = false;
- public static final String BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR = "disableRollingOnLogSegmentError";
- public static final boolean BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR_DEFAULT = false;
-
- // Durability Settings
- public static final String BKDL_IS_DURABLE_WRITE_ENABLED = "isDurableWriteEnabled";
- public static final boolean BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT = true;
-
- // Transmit Settings
- public static final String BKDL_OUTPUT_BUFFER_SIZE = "writerOutputBufferSize";
- public static final String BKDL_OUTPUT_BUFFER_SIZE_OLD = "output-buffer-size";
- public static final int BKDL_OUTPUT_BUFFER_SIZE_DEFAULT = 1024;
- public static final String BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS = "periodicFlushFrequencyMilliSeconds";
- public static final int BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT = 0;
- public static final String BKDL_ENABLE_IMMEDIATE_FLUSH = "enableImmediateFlush";
- public static final boolean BKDL_ENABLE_IMMEDIATE_FLUSH_DEFAULT = false;
- public static final String BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS = "minimumDelayBetweenImmediateFlushMilliSeconds";
- public static final int BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT = 0;
- public static final String BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS = "periodicKeepAliveMilliSeconds";
- public static final int BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS_DEFAULT = 0;
-
- // Retention/Truncation Settings
- public static final String BKDL_RETENTION_PERIOD_IN_HOURS = "logSegmentRetentionHours";
- public static final String BKDL_RETENTION_PERIOD_IN_HOURS_OLD = "retention-size";
- public static final int BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT = 72;
- public static final String BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION = "explicitTruncationByApp";
- public static final boolean BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION_DEFAULT = false;
-
- // Log Segment Rolling Settings
- public static final String BKDL_ROLLING_INTERVAL_IN_MINUTES = "logSegmentRollingMinutes";
- public static final String BKDL_ROLLING_INTERVAL_IN_MINUTES_OLD = "rolling-interval";
- public static final int BKDL_ROLLING_INTERVAL_IN_MINUTES_DEFAULT = 120;
- public static final String BKDL_MAX_LOGSEGMENT_BYTES = "maxLogSegmentBytes";
- public static final int BKDL_MAX_LOGSEGMENT_BYTES_DEFAULT = 256 * 1024 * 1024; // default 256MB
- public static final String BKDL_LOGSEGMENT_ROLLING_CONCURRENCY = "logSegmentRollingConcurrency";
- public static final int BKDL_LOGSEGMENT_ROLLING_CONCURRENCY_DEFAULT = 1;
-
- // Lock Settings
- public static final String BKDL_WRITE_LOCK_ENABLED = "writeLockEnabled";
- public static final boolean BKDL_WRITE_LOCK_ENABLED_DEFAULT = true;
- public static final String BKDL_LOCK_TIMEOUT = "lockTimeoutSeconds";
- public static final long BKDL_LOCK_TIMEOUT_DEFAULT = 30;
- public static final String BKDL_LOCK_REACQUIRE_TIMEOUT = "lockReacquireTimeoutSeconds";
- public static final long BKDL_LOCK_REACQUIRE_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_REACQUIRE_TIMEOUT_DEFAULT;
- public static final String BKDL_LOCK_OP_TIMEOUT = "lockOpTimeoutSeconds";
- public static final long BKDL_LOCK_OP_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_OP_TIMEOUT_DEFAULT;
-
- // Ledger Allocator Settings
- public static final String BKDL_ENABLE_LEDGER_ALLOCATOR_POOL = "enableLedgerAllocatorPool";
- public static final boolean BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT = false;
- public static final String BKDL_LEDGER_ALLOCATOR_POOL_PATH = "ledgerAllocatorPoolPath";
- public static final String BKDL_LEDGER_ALLOCATOR_POOL_PATH_DEFAULT = DistributedLogConstants.ALLOCATION_POOL_NODE;
- public static final String BKDL_LEDGER_ALLOCATOR_POOL_NAME = "ledgerAllocatorPoolName";
- public static final String BKDL_LEDGER_ALLOCATOR_POOL_NAME_DEFAULT = null;
- public static final String BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE = "ledgerAllocatorPoolCoreSize";
- public static final int BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE_DEFAULT = 20;
-
- // Write Limit Settings
- public static final String BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT = "perWriterOutstandingWriteLimit";
- public static final int BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT_DEFAULT = -1;
- public static final String BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT = "globalOutstandingWriteLimit";
- public static final int BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT_DEFAULT = -1;
- public static final String BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE = "outstandingWriteLimitDarkmode";
- public static final boolean BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE_DEFAULT = true;
-
- //
- // DL Reader Settings
- //
-
- // General Settings
- public static final String BKDL_READLAC_OPTION = "readLACLongPoll";
- public static final int BKDL_READLAC_OPTION_DEFAULT = 3; //BKLogPartitionReadHandler.ReadLACOption.READENTRYPIGGYBACK_SEQUENTIAL.value
- public static final String BKDL_READLACLONGPOLL_TIMEOUT = "readLACLongPollTimeout";
- public static final int BKDL_READLACLONGPOLL_TIMEOUT_DEFAULT = 1000;
- public static final String BKDL_DESERIALIZE_RECORDSET_ON_READS = "deserializeRecordSetOnReads";
- public static final boolean BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT = true;
-
- // Idle reader settings
- public static final String BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS = "readerIdleWarnThresholdMillis";
- public static final int BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS_DEFAULT = 120000;
- public static final String BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS = "readerIdleErrorThresholdMillis";
- public static final int BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT = Integer.MAX_VALUE;
-
- // Reader constraint settings
- public static final String BKDL_READER_IGNORE_TRUNCATION_STATUS = "ignoreTruncationStatus";
- public static final boolean BKDL_READER_IGNORE_TRUNCATION_STATUS_DEFAULT = false;
- public static final String BKDL_READER_ALERT_POSITION_ON_TRUNCATED = "alertPositionOnTruncated";
- public static final boolean BKDL_READER_ALERT_POSITION_ON_TRUNCATED_DEFAULT = true;
- public static final String BKDL_READER_POSITION_GAP_DETECTION_ENABLED = "positionGapDetectionEnabled";
- public static final boolean BKDL_READER_POSITION_GAP_DETECTION_ENABLED_DEFAULT = false;
-
- // Read ahead related parameters
- public static final String BKDL_ENABLE_READAHEAD = "enableReadAhead";
- public static final boolean BKDL_ENABLE_READAHEAD_DEFAULT = true;
- public static final String BKDL_ENABLE_FORCEREAD = "enableForceRead";
- public static final boolean BKDL_ENABLE_FORCEREAD_DEFAULT = true;
- public static final String BKDL_READAHEAD_MAX_RECORDS = "readAheadMaxRecords";
- public static final String BKDL_READAHEAD_MAX_RECORDS_OLD = "ReadAheadMaxEntries";
- public static final int BKDL_READAHEAD_MAX_RECORDS_DEFAULT = 10;
- public static final String BKDL_READAHEAD_BATCHSIZE = "readAheadBatchSize";
- public static final String BKDL_READAHEAD_BATCHSIZE_OLD = "ReadAheadBatchSize";
- public static final int BKDL_READAHEAD_BATCHSIZE_DEFAULT = 2;
- public static final String BKDL_READAHEAD_WAITTIME = "readAheadWaitTime";
- public static final String BKDL_READAHEAD_WAITTIME_OLD = "ReadAheadWaitTime";
- public static final int BKDL_READAHEAD_WAITTIME_DEFAULT = 200;
- public static final String BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM = "readAheadWaitTimeOnEndOfStream";
- public static final String BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_OLD = "ReadAheadWaitTimeOnEndOfStream";
- public static final int BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_DEFAULT = 10000;
- public static final String BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS =
- "readAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis";
- public static final int BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS_DEFAULT = 10000;
- public static final String BKDL_READAHEAD_SKIP_BROKEN_ENTRIES = "readAheadSkipBrokenEntries";
- public static final boolean BKDL_READAHEAD_SKIP_BROKEN_ENTRIES_DEFAULT = false;
- public static final String BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT = "numPrefetchEntriesPerLogSegment";
- public static final int BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT = 4;
- public static final String BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT = "maxPrefetchEntriesPerLogSegment";
- public static final int BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT = 32;
-
- // Scan Settings
- public static final String BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "firstNumEntriesEachPerLastRecordScan";
- public static final int BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 2;
- public static final String BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "maxNumEntriesPerReadLastRecordScan";
- public static final int BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 16;
-
- // Log Existence Settings
- public static final String BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS = "checkLogExistenceBackoffStartMillis";
- public static final int BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS_DEFAULT = 200;
- public static final String BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS = "checkLogExistenceBackoffMaxMillis";
- public static final int BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS_DEFAULT = 1000;
-
- //
- // Tracing/Stats Settings
- //
-
- public static final String BKDL_TRACE_READAHEAD_DELIVERY_LATENCY = "traceReadAheadDeliveryLatency";
- public static final boolean BKDL_TRACE_READAHEAD_DELIVERY_LATENCY_DEFAULT = false;
- public static final String BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS = "metadataLatencyWarnThresholdMs";
- public static final long BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
- public static final String BKDL_DATA_LATENCY_WARN_THRESHOLD_MS = "dataLatencyWarnThresholdMs";
- public static final long BKDL_DATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = 2 * DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
- public static final String BKDL_TRACE_READAHEAD_METADATA_CHANGES = "traceReadAheadMetadataChanges";
- public static final boolean BKDL_TRACE_READAHEAD_MEATDATA_CHANGES_DEFAULT = false;
- public final static String BKDL_ENABLE_TASK_EXECUTION_STATS = "enableTaskExecutionStats";
- public final static boolean BKDL_ENABLE_TASK_EXECUTION_STATS_DEFAULT = false;
- public final static String BKDL_TASK_EXECUTION_WARN_TIME_MICROS = "taskExecutionWarnTimeMicros";
- public final static long BKDL_TASK_EXECUTION_WARN_TIME_MICROS_DEFAULT = 100000;
- public static final String BKDL_ENABLE_PERSTREAM_STAT = "enablePerStreamStat";
- public static final boolean BKDL_ENABLE_PERSTREAM_STAT_DEFAULT = false;
-
- //
- // Settings for Feature Providers
- //
-
- public static final String BKDL_FEATURE_PROVIDER_CLASS = "featureProviderClass";
-
- //
- // Settings for Configuration Based Feature Provider
- //
-
- public static final String BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH = "fileFeatureProviderBaseConfigPath";
- public static final String BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH_DEFAULT = "decider.conf";
- public static final String BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH = "fileFeatureProviderOverlayConfigPath";
- public static final String BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH_DEFAULT = null;
-
- //
- // Settings for Namespaces
- //
-
- public static final String BKDL_FEDERATED_NAMESPACE_ENABLED = "federatedNamespaceEnabled";
- public static final boolean BKDL_FEDERATED_NAMESPACE_ENABLED_DEFAULT = false;
- public static final String BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE = "federatedMaxLogsPerSubnamespace";
- public static final int BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE_DEFAULT = 15000;
- public static final String BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS = "federatedCheckExistenceWhenCacheMiss";
- public static final boolean BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS_DEFAULT = true;
-
- // Settings for Configurations
-
- public static final String BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC = "dynamicConfigReloadIntervalSec";
- public static final int BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC_DEFAULT = 60;
- public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS = "streamConfigRouterClass";
- public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS_DEFAULT = "com.twitter.distributedlog.service.config.IdentityConfigRouter";
-
- // Settings for RateLimit (used by distributedlog-service)
-
- public static final String BKDL_BPS_SOFT_WRITE_LIMIT = "bpsSoftWriteLimit";
- public static final int BKDL_BPS_SOFT_WRITE_LIMIT_DEFAULT = -1;
- public static final String BKDL_BPS_HARD_WRITE_LIMIT = "bpsHardWriteLimit";
- public static final int BKDL_BPS_HARD_WRITE_LIMIT_DEFAULT = -1;
- public static final String BKDL_RPS_SOFT_WRITE_LIMIT = "rpsSoftWriteLimit";
- public static final int BKDL_RPS_SOFT_WRITE_LIMIT_DEFAULT = -1;
- public static final String BKDL_RPS_HARD_WRITE_LIMIT = "rpsHardWriteLimit";
- public static final int BKDL_RPS_HARD_WRITE_LIMIT_DEFAULT = -1;
-
- // Rate and resource limits: per shard
-
- public static final String BKDL_RPS_SOFT_SERVICE_LIMIT = "rpsSoftServiceLimit";
- public static final int BKDL_RPS_SOFT_SERVICE_LIMIT_DEFAULT = -1;
- public static final String BKDL_RPS_HARD_SERVICE_LIMIT = "rpsHardServiceLimit";
- public static final int BKDL_RPS_HARD_SERVICE_LIMIT_DEFAULT = -1;
- public static final String BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT = "rpsStreamAcquireServiceLimit";
- public static final int BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT = -1;
- public static final String BKDL_BPS_SOFT_SERVICE_LIMIT = "bpsSoftServiceLimit";
- public static final int BKDL_BPS_SOFT_SERVICE_LIMIT_DEFAULT = -1;
- public static final String BKDL_BPS_HARD_SERVICE_LIMIT = "bpsHardServiceLimit";
- public static final int BKDL_BPS_HARD_SERVICE_LIMIT_DEFAULT = -1;
- public static final String BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT = "bpsStreamAcquireServiceLimit";
- public static final int BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT = -1;
-
- // Settings for Partitioning
-
- public static final String BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY = "maxAcquiredPartitionsPerProxy";
- public static final int BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT = -1;
-
- public static final String BKDL_MAX_CACHED_PARTITIONS_PER_PROXY = "maxCachedPartitionsPerProxy";
- public static final int BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT = -1;
-
- //
- // Settings for Error Injection
- //
- public static final String BKDL_EI_INJECT_WRITE_DELAY = "eiInjectWriteDelay";
- public static final boolean BKDL_EI_INJECT_WRITE_DELAY_DEFAULT = false;
- public static final String BKDL_EI_INJECTED_WRITE_DELAY_PERCENT = "eiInjectedWriteDelayPercent";
- public static final double BKDL_EI_INJECTED_WRITE_DELAY_PERCENT_DEFAULT = 0.0;
- public static final String BKDL_EI_INJECTED_WRITE_DELAY_MS = "eiInjectedWriteDelayMs";
- public static final int BKDL_EI_INJECTED_WRITE_DELAY_MS_DEFAULT = 0;
- public static final String BKDL_EI_INJECT_READAHEAD_STALL = "eiInjectReadAheadStall";
- public static final boolean BKDL_EI_INJECT_READAHEAD_STALL_DEFAULT = false;
- public static final String BKDL_EI_INJECT_READAHEAD_DELAY = "eiInjectReadAheadDelay";
- public static final boolean BKDL_EI_INJECT_READAHEAD_DELAY_DEFAULT = false;
- public static final String BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS = "eiInjectMaxReadAheadDelayMs";
- public static final int BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS_DEFAULT = 0;
- public static final String BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT = "eiInjectReadAheadDelayPercent";
- public static final int BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT_DEFAULT = 10;
- public static final String BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES = "eiInjectReadAheadBrokenEntries";
- public static final boolean BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES_DEFAULT = false;
-
- // Whitelisted stream-level configuration settings.
- private static final Set<String> streamSettings = Sets.newHashSet(
- BKDL_READER_POSITION_GAP_DETECTION_ENABLED,
- BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS,
- BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS,
- BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
- BKDL_ENABLE_IMMEDIATE_FLUSH
- );
-
- /**
- * Construct distributedlog configuration with default settings.
- * It also loads the settings from system properties.
- */
- public DistributedLogConfiguration() {
- super();
- // add configuration for system properties
- addConfiguration(new SystemConfiguration());
- }
-
- /**
- * You can load configurations in precedence order. The first one takes
- * precedence over any loaded later.
- *
- * @param confURL Configuration URL
- */
- public void loadConf(URL confURL) throws ConfigurationException {
- Configuration loadedConf = new PropertiesConfiguration(confURL);
- addConfiguration(loadedConf);
- }
-
- /**
- * You can load configuration from other configuration
- *
- * @param baseConf Other Configuration
- */
- public void loadConf(DistributedLogConfiguration baseConf) {
- addConfiguration(baseConf);
- }
-
- /**
- * Load configuration from other configuration object
- *
- * @param otherConf Other configuration object
- */
- public void loadConf(Configuration otherConf) {
- addConfiguration(otherConf);
- }
-
- /**
- * Load whitelisted stream configuration from another configuration object
- *
- * @param streamConfiguration stream configuration overrides
- */
- public void loadStreamConf(Optional<DistributedLogConfiguration> streamConfiguration) {
- if (!streamConfiguration.isPresent()) {
- return;
- }
- ArrayList<Object> ignoredSettings = new ArrayList<Object>();
- Iterator iterator = streamConfiguration.get().getKeys();
- while (iterator.hasNext()) {
- Object setting = iterator.next();
- if (setting instanceof String && streamSettings.contains(setting)) {
- String settingStr = (String) setting;
- setProperty(settingStr, streamConfiguration.get().getProperty(settingStr));
- } else {
- ignoredSettings.add(setting);
- }
- }
- if (LOG.isWarnEnabled() && !ignoredSettings.isEmpty()) {
- LOG.warn("invalid stream configuration override(s): {}",
- StringUtils.join(ignoredSettings, ";"));
- }
- }
-
- //
- // ZooKeeper Related Settings
- //
-
- /**
- * Get all properties as a string.
- */
- public String getPropsAsString() {
- Iterator iterator = getKeys();
- StringBuilder builder = new StringBuilder();
- boolean appendNewline = false;
- while (iterator.hasNext()) {
- Object key = iterator.next();
- if (key instanceof String) {
- if (appendNewline) {
- builder.append("\n");
- }
- Object value = getProperty((String)key);
- builder.append(key).append("=").append(value);
- appendNewline = true;
- }
- }
- return builder.toString();
- }
-
- /**
- * Get digest id used for ZK acl.
- *
- * @return zk acl id.
- */
- public String getZkAclId() {
- return getString(BKDL_ZK_ACL_ID, BKDL_ZK_ACL_ID_DEFAULT);
- }
-
- /**
- * Set digest id to use for ZK acl.
- *
- * @param zkAclId acl id.
- * @return distributedlog configuration
- * @see #getZkAclId()
- */
- public DistributedLogConfiguration setZkAclId(String zkAclId) {
- setProperty(BKDL_ZK_ACL_ID, zkAclId);
- return this;
- }
-
- /**
- * Get ZK Session timeout in seconds.
- * <p>
- * This is the session timeout applied for zookeeper client used by distributedlog.
- * Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
- * by bookkeeper client.
- *
- * @return zookeeeper session timeout in seconds.
- * @deprecated use {@link #getZKSessionTimeoutMilliseconds()}
- */
- public int getZKSessionTimeoutSeconds() {
- return this.getInt(BKDL_ZK_SESSION_TIMEOUT_SECONDS, BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT);
- }
-
- /**
- * Get ZK Session timeout in milliseconds.
- * <p>
- * This is the session timeout applied for zookeeper client used by distributedlog.
- * Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
- * by bookkeeper client.
- *
- * @return zk session timeout in milliseconds.
- */
- public int getZKSessionTimeoutMilliseconds() {
- return this.getInt(BKDL_ZK_SESSION_TIMEOUT_SECONDS, BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT) * 1000;
- }
-
- /**
- * Set ZK Session Timeout in seconds.
- *
- * @param zkSessionTimeoutSeconds session timeout in seconds.
- * @return distributed log configuration
- * @see #getZKSessionTimeoutMilliseconds()
- */
- public DistributedLogConfiguration setZKSessionTimeoutSeconds(int zkSessionTimeoutSeconds) {
- setProperty(BKDL_ZK_SESSION_TIMEOUT_SECONDS, zkSessionTimeoutSeconds);
- return this;
- }
-
- /**
- * Get zookeeper access rate limit.
- * <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
- * It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
- * the rate limiting is disable. By default it is disable (value = 0).
- *
- * @return zookeeper access rate, by default it is 0.
- */
- public double getZKRequestRateLimit() {
- return this.getDouble(BKDL_ZK_REQUEST_RATE_LIMIT, BKDL_ZK_REQUEST_RATE_LIMIT_DEFAULT);
- }
-
- /**
- * Set zookeeper access rate limit (rps).
- *
- * @param requestRateLimit
- * zookeeper access rate limit
- * @return distributedlog configuration
- * @see #getZKRequestRateLimit()
- */
- public DistributedLogConfiguration setZKRequestRateLimit(double requestRateLimit) {
- setProperty(BKDL_ZK_REQUEST_RATE_LIMIT, requestRateLimit);
- return this;
- }
-
- /**
- * Get num of retries per request for zookeeper client.
- * <p>Retries only happen on retryable failures like session expired,
- * session moved. for permanent failures, the request will fail immediately.
- * The default value is 3.
- *
- * @return num of retries per request of zookeeper client.
- */
- public int getZKNumRetries() {
- return this.getInt(BKDL_ZK_NUM_RETRIES, BKDL_ZK_NUM_RETRIES_DEFAULT);
- }
-
- /**
- * Set num of retries per request for zookeeper client.
- *
- * @param zkNumRetries num of retries per request of zookeeper client.
- * @return distributed log configuration
- * @see #getZKNumRetries()
- */
- public DistributedLogConfiguration setZKNumRetries(int zkNumRetries) {
- setProperty(BKDL_ZK_NUM_RETRIES, zkNumRetries);
- return this;
- }
-
- /**
- * Get the start backoff time of zookeeper operation retries, in milliseconds.
- * <p>The retry time will increase in bound exponential way, and become flat
- * after hit max backoff time ({@link #getZKRetryBackoffMaxMillis()}).
- * The default start backoff time is 5000 milliseconds.
- *
- * @return start backoff time of zookeeper operation retries, in milliseconds.
- * @see #getZKRetryBackoffMaxMillis()
- */
- public int getZKRetryBackoffStartMillis() {
- return this.getInt(BKDL_ZK_RETRY_BACKOFF_START_MILLIS,
- BKDL_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT);
- }
-
- /**
- * Set the start backoff time of zookeeper operation retries, in milliseconds.
- *
- * @param zkRetryBackoffStartMillis start backoff time of zookeeper operation retries,
- * in milliseconds.
- * @return distributed log configuration
- * @see #getZKRetryBackoffStartMillis()
- */
- public DistributedLogConfiguration setZKRetryBackoffStartMillis(int zkRetryBackoffStartMillis) {
- setProperty(BKDL_ZK_RETRY_BACKOFF_START_MILLIS, zkRetryBackoffStartMillis);
- return this;
- }
-
- /**
- * Get the max backoff time of zookeeper operation retries, in milliseconds.
- * <p>The retry time will increase in bound exponential way starting from
- * {@link #getZKRetryBackoffStartMillis()}, and become flat after hit this max
- * backoff time.
- * The default max backoff time is 30000 milliseconds.
- *
- * @return max backoff time of zookeeper operation retries, in milliseconds.
- * @see #getZKRetryBackoffStartMillis()
- */
- public int getZKRetryBackoffMaxMillis() {
- return this.getInt(BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS,
- BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT);
- }
-
- /**
- * Set the max backoff time of zookeeper operation retries, in milliseconds.
- *
- * @param zkRetryBackoffMaxMillis max backoff time of zookeeper operation retries,
- * in milliseconds.
- * @return distributed log configuration
- * @see #getZKRetryBackoffMaxMillis()
- */
- public DistributedLogConfiguration setZKRetryBackoffMaxMillis(int zkRetryBackoffMaxMillis) {
- setProperty(BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS, zkRetryBackoffMaxMillis);
- return this;
- }
-
- /**
- * Get ZK client number of retry executor threads.
- * By default it is 1.
- *
- * @return number of bookkeeper client worker threads.
- */
- public int getZKClientNumberRetryThreads() {
- return this.getInt(BKDL_ZKCLIENT_NUM_RETRY_THREADS, BKDL_ZKCLIENT_NUM_RETRY_THREADS_DEFAULT);
- }
-
- /**
- * Set ZK client number of retry executor threads.
- *
- * @param numThreads
- * number of retry executor threads.
- * @return distributedlog configuration.
- * @see #getZKClientNumberRetryThreads()
- */
- public DistributedLogConfiguration setZKClientNumberRetryThreads(int numThreads) {
- setProperty(BKDL_ZKCLIENT_NUM_RETRY_THREADS, numThreads);
- return this;
- }
-
- //
- // BookKeeper ZooKeeper Client Settings
- //
-
- /**
- * Get BK's zookeeper session timout in milliseconds.
- * <p>
- * This is the session timeout applied for zookeeper client used by bookkeeper client.
- * Use {@link #getZKSessionTimeoutMilliseconds()} for zookeeper client used
- * by distributedlog.
- *
- * @return Bk's zookeeper session timeout in milliseconds
- */
- public int getBKClientZKSessionTimeoutMilliSeconds() {
- return this.getInt(BKDL_BKCLIENT_ZK_SESSION_TIMEOUT, BKDL_BKCLIENT_ZK_SESSION_TIMEOUT_DEFAULT) * 1000;
- }
-
- /**
- * Set BK's zookeeper session timeout in seconds.
- *
- * @param sessionTimeout session timeout for the ZK Client used by BK Client, in seconds.
- * @return distributed log configuration
- * @see #getBKClientZKSessionTimeoutMilliSeconds()
- */
- public DistributedLogConfiguration setBKClientZKSessionTimeout(int sessionTimeout) {
- setProperty(BKDL_BKCLIENT_ZK_SESSION_TIMEOUT, sessionTimeout);
- return this;
- }
-
- /**
- * Get zookeeper access rate limit for zookeeper client used in bookkeeper client.
- * <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
- * It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
- * the rate limiting is disable. By default it is disable (value = 0).
- *
- * @return zookeeper access rate limit for zookeeper client used in bookkeeper client.
- * By default it is 0.
- */
- public double getBKClientZKRequestRateLimit() {
- return this.getDouble(BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT,
- BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT_DEFAULT);
- }
-
- /**
- * Set zookeeper access rate limit for zookeeper client used in bookkeeper client.
- *
- * @param rateLimit
- * zookeeper access rate limit
- * @return distributedlog configuration.
- * @see #getBKClientZKRequestRateLimit()
- */
- public DistributedLogConfiguration setBKClientZKRequestRateLimit(double rateLimit) {
- setProperty(BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT, rateLimit);
- return this;
- }
-
- /**
- * Get num of retries for zookeeper client that used by bookkeeper client.
- * <p>Retries only happen on retryable failures like session expired,
- * session moved. for permanent failures, the request will fail immediately.
- * The default value is 3. Setting it to zero or negative will retry infinitely.
- *
- * @return num of retries of zookeeper client used by bookkeeper client.
- */
- public int getBKClientZKNumRetries() {
- int zkNumRetries = this.getInt(BKDL_BKCLIENT_ZK_NUM_RETRIES, BKDL_BKCLIENT_ZK_NUM_RETRIES_DEFAULT);
- if (zkNumRetries <= 0) {
- return Integer.MAX_VALUE;
- }
- return zkNumRetries;
- }
-
- /**
- * Get the start backoff time of zookeeper operation retries, in milliseconds.
- * <p>The retry time will increase in bound exponential way, and become flat
- * after hit max backoff time ({@link #getBKClientZKRetryBackoffMaxMillis()}.
- * The default start backoff time is 5000 milliseconds.
- *
- * @return start backoff time of zookeeper operation retries, in milliseconds.
- * @see #getBKClientZKRetryBackoffMaxMillis()
- */
- public int getBKClientZKRetryBackoffStartMillis() {
- return this.getInt(BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS,
- BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT);
- }
-
- /**
- * Get the max backoff time of zookeeper operation retries, in milliseconds.
- * <p>The retry time will increase in bound exponential way starting from
- * {@link #getBKClientZKRetryBackoffStartMillis()}, and become flat after
- * hit this max backoff time.
- * The default max backoff time is 30000 milliseconds.
- *
- * @return max backoff time of zookeeper operation retries, in milliseconds.
- * @see #getBKClientZKRetryBackoffStartMillis()
- */
- public int getBKClientZKRetryBackoffMaxMillis() {
- return this.getInt(BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS,
- BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT);
- }
-
- //
- // BookKeeper Ensemble Placement Settings
- //
-
- /**
- * Get ensemble size of each log segment (ledger) will use.
- * By default it is 3.
- * <p>
- * A log segment's data is stored in an ensemble of bookies in
- * a stripping way. Each entry will be added in a <code>write-quorum</code>
- * size of bookies. The add operation will complete once it receives
- * responses from a <code>ack-quorum</code> size of bookies. The stripping
- * is done in a round-robin way in bookkeeper.
- * <p>
- * For example, we configure the ensemble-size to 5, write-quorum-size to 3,
- * and ack-quorum-size to 2. The data will be stored in following stripping way.
- * <pre>
- * | entry id | bk1 | bk2 | bk3 | bk4 | bk5 |
- * | 0 | x | x | x | | |
- * | 1 | | x | x | x | |
- * | 2 | | | x | x | x |
- * | 3 | x | | | x | x |
- * | 4 | x | x | | | x |
- * | 5 | x | x | x | | |
- * </pre>
- * <p>
- * We don't recommend stripping within a log segment to increase bandwidth.
- * We'd recommend to strip by `partition` in higher level of distributedlog
- * to increase performance. so typically the ensemble size will set to be
- * the same value as write quorum size.
- *
- * @return ensemble size
- * @see #getWriteQuorumSize()
- * @see #getAckQuorumSize()
- */
- public int getEnsembleSize() {
- return this.getInt(BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
- getInt(BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
- BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT));
- }
-
- /**
- * Set ensemble size of each log segment (ledger) will use.
- *
- * @param ensembleSize ensemble size.
- * @return distributed log configuration
- * @see #getEnsembleSize()
- */
- public DistributedLogConfiguration setEnsembleSize(int ensembleSize) {
- setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE, ensembleSize);
- return this;
- }
-
- /**
- * Get write quorum size of each log segment (ledger) will use.
- * By default it is 3.
- *
- * @return write quorum size
- * @see #getEnsembleSize()
- */
- public int getWriteQuorumSize() {
- return this.getInt(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
- getInt(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
- BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT));
- }
-
- /**
- * Set write quorum size of each log segment (ledger) will use.
- *
- * @param quorumSize
- * quorum size.
- * @return distributedlog configuration.
- * @see #getWriteQuorumSize()
- */
- public DistributedLogConfiguration setWriteQuorumSize(int quorumSize) {
- setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE, quorumSize);
- return this;
- }
-
- /**
- * Get ack quorum size of each log segment (ledger) will use.
- * By default it is 2.
- *
- * @return ack quorum size
- * @see #getEnsembleSize()
- */
- public int getAckQuorumSize() {
- return this.getInt(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
- getInt(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
- BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT));
- }
-
- /**
- * Set ack quorum size of each log segment (ledger) will use.
- *
- * @param quorumSize
- * quorum size.
- * @return distributedlog configuration.
- * @see #getAckQuorumSize()
- */
- public DistributedLogConfiguration setAckQuorumSize(int quorumSize) {
- setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE, quorumSize);
- return this;
- }
-
- /**
- * Get the quorum config for each log segment (ledger).
- *
- * @return quorum config that used by log segments
- * @see #getEnsembleSize()
- * @see #getWriteQuorumSize()
- * @see #getAckQuorumSize()
- */
- public QuorumConfig getQuorumConfig() {
- return new QuorumConfig(
- getEnsembleSize(),
- getWriteQuorumSize(),
- getAckQuorumSize());
- }
-
- /**
- * Get if row aware ensemble placement is enabled.
- * <p>If enabled, {@link DNSResolverForRows} will be used for dns resolution
- * rather than {@link DNSResolverForRacks}, if no other dns resolver set via
- * {@link #setEnsemblePlacementDnsResolverClass(Class)}.
- * By default it is disable.
- *
- * @return true if row aware ensemble placement is enabled, otherwise false.
- * @see #getEnsemblePlacementDnsResolverClass()
- */
- public boolean getRowAwareEnsemblePlacementEnabled() {
- return getBoolean(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT,
- getBoolean(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_OLD,
- BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_DEFAULT));
- }
-
- /**
- * Set if we should enable row aware ensemble placement.
- *
- * @param enableRowAwareEnsemblePlacement
- * enableRowAwareEnsemblePlacement
- * @return distributedlog configuration.
- * @see #getRowAwareEnsemblePlacementEnabled()
- */
- public DistributedLogConfiguration setRowAwareEnsemblePlacementEnabled(boolean enableRowAwareEnsemblePlacement) {
- setProperty(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT, enableRowAwareEnsemblePlacement);
- return this;
- }
-
- /**
- * Get the DNS resolver class for bookkeeper ensemble placement.
- * <p>By default, {@link DNSResolverForRacks} will be used if
- * {@link #getRowAwareEnsemblePlacementEnabled()} is disabled and
- * {@link DNSResolverForRows} will be used if {@link #getRowAwareEnsemblePlacementEnabled()}
- * is enabled.
- *
- * @return dns resolver class for bookkeeper ensemble placement.
- * @throws ConfigurationException
- * @see #getRowAwareEnsemblePlacementEnabled()
- */
- public Class<? extends DNSToSwitchMapping> getEnsemblePlacementDnsResolverClass()
- throws ConfigurationException {
- Class<? extends DNSToSwitchMapping> defaultResolverCls;
- if (getRowAwareEnsemblePlacementEnabled()) {
- defaultResolverCls = DNSResolverForRows.class;
- } else {
- defaultResolverCls = DNSResolverForRacks.class;
- }
- return ReflectionUtils.getClass(this, BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS,
- defaultResolverCls, DNSToSwitchMapping.class, defaultLoader);
- }
-
- /**
- * Set the DNS resolver class for bookkeeper ensemble placement.
- *
- * @param dnsResolverClass
- * dns resolver class for bookkeeper ensemble placement.
- * @return distributedlog configuration
- * @see #getEnsemblePlacementDnsResolverClass()
- */
- public DistributedLogConfiguration setEnsemblePlacementDnsResolverClass(
- Class<? extends DNSToSwitchMapping> dnsResolverClass) {
- setProperty(BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS, dnsResolverClass.getName());
- return this;
- }
-
- /**
- * Get mapping used to override the region mapping derived by the default resolver.
- * <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
- * By default it is empty string.
- *
- * @return dns resolver overrides.
- * @see #getEnsemblePlacementDnsResolverClass()
- * @see DNSResolverForRacks
- * @see DNSResolverForRows
- */
- public String getBkDNSResolverOverrides() {
- return getString(BKDL_BK_DNS_RESOLVER_OVERRIDES, BKDL_BK_DNS_RESOLVER_OVERRIDES_DEFAULT);
- }
-
- /**
- * Set mapping used to override the region mapping derived by the default resolver
- * <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
- * By default it is empty string.
- *
- * @param overrides
- * dns resolver overrides
- * @return dl configuration.
- * @see #getBkDNSResolverOverrides()
- */
- public DistributedLogConfiguration setBkDNSResolverOverrides(String overrides) {
- setProperty(BKDL_BK_DNS_RESOLVER_OVERRIDES, overrides);
- return this;
- }
-
- //
- // BookKeeper General Settings
- //
-
- /**
- * Set password used by bookkeeper client for digestion.
- * <p>
- * NOTE: not recommend to change. will be derepcated in future.
- *
- * @param bkDigestPW BK password digest
- * @return distributedlog configuration
- */
- public DistributedLogConfiguration setBKDigestPW(String bkDigestPW) {
- setProperty(BKDL_BOOKKEEPER_DIGEST_PW, bkDigestPW);
- return this;
- }
-
- /**
- * Get password used by bookkeeper client for digestion.
- * <p>
- * NOTE: not recommend to change. will be deprecated in future.
- *
- * @return password used by bookkeeper client for digestion
- * @see #setBKDigestPW(String)
- */
- public String getBKDigestPW() {
- return getString(BKDL_BOOKKEEPER_DIGEST_PW, BKDL_BOOKKEEPER_DIGEST_PW_DEFAULT);
- }
-
- /**
- * Get BK client number of i/o threads used by Netty.
- * The default value equals DL's number worker threads.
- *
- * @return number of bookkeeper netty i/o threads.
- * @see #getNumWorkerThreads()
- */
- public int getBKClientNumberIOThreads() {
- return this.getInt(BKDL_BKCLIENT_NUM_IO_THREADS, getNumWorkerThreads());
- }
-
- /**
- * Set BK client number of i/o threads used by netty.
- *
- * @param numThreads
- * number io threads.
- * @return distributedlog configuration.
- * @see #getBKClientNumberIOThreads()
- */
- public DistributedLogConfiguration setBKClientNumberIOThreads(int numThreads) {
- setProperty(BKDL_BKCLIENT_NUM_IO_THREADS, numThreads);
- return this;
- }
-
- /**
- * Get the tick duration in milliseconds that used for timeout timer in bookkeeper client.
- * By default it is 100.
- *
- * @return tick duration in milliseconds
- * @see org.jboss.netty.util.HashedWheelTimer
- */
- public long getTimeoutTimerTickDurationMs() {
- return getLong(BKDL_TIMEOUT_TIMER_TICK_DURATION_MS, BKDL_TIMEOUT_TIMER_TICK_DURATION_MS_DEFAULT);
- }
-
- /**
- * Set the tick duration in milliseconds that used for timeout timer in bookkeeper client.
- *
- * @param tickDuration
- * tick duration in milliseconds.
- * @return distributed log configuration.
- * @see #getTimeoutTimerTickDurationMs()
- */
- public DistributedLogConfiguration setTimeoutTimerTickDurationMs(long tickDuration) {
- setProperty(BKDL_TIMEOUT_TIMER_TICK_DURATION_MS, tickDuration);
- return this;
- }
-
- /**
- * Get number of ticks that used for timeout timer in bookkeeper client.
- * By default is 1024.
- *
- * @return number of ticks that used for timeout timer.
- * @see org.jboss.netty.util.HashedWheelTimer
- */
- public int getTimeoutTimerNumTicks() {
- return getInt(BKDL_TIMEOUT_TIMER_NUM_TICKS, BKDL_TIMEOUT_TIMER_NUM_TICKS_DEFAULT);
- }
-
- /**
- * Set number of ticks that used for timeout timer in bookkeeper client.
- *
- * @param numTicks
- * number of ticks that used for timeout timer.
- * @return distributed log configuration.
- * @see #getTimeoutTimerNumTicks()
- */
- public DistributedLogConfiguration setTimeoutTimerNumTicks(int numTicks) {
- setProperty(BKDL_TIMEOUT_TIMER_NUM_TICKS, numTicks);
- return this;
- }
-
- //
- // Deprecated BookKeeper Settings
- //
-
- /**
- * Get BK client read timeout in seconds.
- * <p>
- * Please use {@link ClientConfiguration#getReadEntryTimeout()}
- * instead of this setting.
- *
- * @return read timeout in seconds
- * @deprecated
- * @see ClientConfiguration#getReadEntryTimeout()
- */
- public int getBKClientReadTimeout() {
- return this.getInt(BKDL_BKCLIENT_READ_TIMEOUT,
- BKDL_BKCLIENT_READ_TIMEOUT_DEFAULT);
- }
-
- /**
- * Set BK client read timeout in seconds.
- *
- * @param readTimeout read timeout in seconds.
- * @return distributed log configuration
- * @deprecated
- * @see #getBKClientReadTimeout()
- */
- public DistributedLogConfiguration setBKClientReadTimeout(int readTimeout) {
- setProperty(BKDL_BKCLIENT_READ_TIMEOUT, readTimeout);
- return this;
- }
-
- /**
- * Get BK client write timeout in seconds.
- * <p>
- * Please use {@link ClientConfiguration#getAddEntryTimeout()}
- * instead of this setting.
- *
- * @return write timeout in seconds.
- * @deprecated
- * @see ClientConfiguration#getAddEntryTimeout()
- */
- public int getBKClientWriteTimeout() {
- return this.getInt(BKDL_BKCLIENT_WRITE_TIMEOUT, BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
- }
-
- /**
- * Set BK client write timeout in seconds
- *
- * @param writeTimeout write timeout in seconds.
- * @return distributed log configuration
- * @deprecated
- * @see #getBKClientWriteTimeout()
- */
- public DistributedLogConfiguration setBKClientWriteTimeout(int writeTimeout) {
- setProperty(BKDL_BKCLIENT_WRITE_TIMEOUT, writeTimeout);
- return this;
- }
-
- /**
- * Get BK client number of worker threads.
- * <p>
- * Please use {@link ClientConfiguration#getNumWorkerThreads()}
- * instead of this setting.
- *
- * @return number of bookkeeper client worker threads.
- * @deprecated
- * @see ClientConfiguration#getNumWorkerThreads()
- */
- public int getBKClientNumberWorkerThreads() {
- return this.getInt(BKDL_BKCLIENT_NUM_WORKER_THREADS, BKDL_BKCLEINT_NUM_WORKER_THREADS_DEFAULT);
- }
-
- /**
- * Set BK client number of worker threads.
- *
- * @param numThreads
- * number worker threads.
- * @return distributedlog configuration.
- * @deprecated
- * @see #getBKClientNumberWorkerThreads()
- */
- public DistributedLogConfiguration setBKClientNumberWorkerThreads(int numThreads) {
- setProperty(BKDL_BKCLIENT_NUM_WORKER_THREADS, numThreads);
- return this;
- }
-
- //
- // DL Executor Settings
- //
-
- /**
- * Get the number of worker threads used by distributedlog namespace.
- * By default it is the number of available processors.
- *
- * @return number of worker threads used by distributedlog namespace.
- */
- public int getNumWorkerThreads() {
- return getInt(BKDL_NUM_WORKER_THREADS, Runtime.getRuntime().availableProcessors());
- }
-
- /**
- * Set the number of worker threads used by distributedlog namespace.
- *
- * @param numWorkerThreads
- * number of worker threads used by distributedlog namespace.
- * @return configuration
- * @see #getNumWorkerThreads()
- */
- public DistributedLogConfiguration setNumWorkerThreads(int numWorkerThreads) {
- setProperty(BKDL_NUM_WORKER_THREADS, numWorkerThreads);
- return this;
- }
-
- /**
- * Get the number of dedicated readahead worker threads used by distributedlog namespace.
- * <p>If this value is non-positive, it would share the normal executor (see {@link #getNumWorkerThreads()}
- * for readahead. otherwise, it would use a dedicated executor for readhead. By default,
- * it is 0.
- *
- * @return number of dedicated readahead worker threads.
- * @see #getNumWorkerThreads()
- */
- @Deprecated
- public int getNumReadAheadWorkerThreads() {
- return getInt(BKDL_NUM_READAHEAD_WORKER_THREADS, 0);
- }
-
- /**
- * Set the number of dedicated readahead worker threads used by distributedlog namespace.
- *
- * @param numWorkerThreads
- * number of dedicated readahead worker threads.
- * @return configuration
- * @see #getNumReadAheadWorkerThreads()
- */
- @Deprecated
- public DistributedLogConfiguration setNumReadAheadWorkerThreads(int numWorkerThreads) {
- setProperty(BKDL_NUM_READAHEAD_WORKER_THREADS, numWorkerThreads);
- return this;
- }
-
- /**
- * Get the number of lock state threads used by distributedlog namespace.
- * By default it is 1.
- *
- * @return number of lock state threads used by distributedlog namespace.
- */
- public int getNumLockStateThreads() {
- return getInt(BKDL_NUM_LOCKSTATE_THREADS, 1);
- }
-
- /**
- * Set the number of lock state threads used by distributedlog manager factory.
- *
- * @param numLockStateThreads
- * number of lock state threads used by distributedlog manager factory.
- * @return configuration
- * @see #getNumLockStateThreads()
- */
- public DistributedLogConfiguration setNumLockStateThreads(int numLockStateThreads) {
- setProperty(BKDL_NUM_LOCKSTATE_THREADS, numLockStateThreads);
- return this;
- }
-
- /**
- * Get the number of resource release threads used by distributedlog namespace.
- * By default it is 0 - the thread will be created dynamically by a executor service.
- * The executor service is an unbounded pool. Application can use `total_tasks - completed_tasks`
- * on monitoring the number of threads that are used for releasing resources.
- * <p>
- * The setting is only applied for v2 implementation.
- *
- * @see com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor
- * @return number of resource release threads used by distributedlog namespace.
- */
- public int getNumResourceReleaseThreads() {
- return getInt(BKDL_NUM_RESOURCE_RELEASE_THREADS, 0);
- }
-
- /**
- * Set the number of resource release threads used by distributedlog manager factory.
- *
- * @param numResourceReleaseThreads
- * number of resource release threads used by distributedlog manager factory.
- * @return configuration
- * @see #getNumResourceReleaseThreads()
- */
- public DistributedLogConfiguration setNumResourceReleaseThreads(int numResourceReleaseThreads) {
- setProperty(BKDL_NUM_RESOURCE_RELEASE_THREADS, numResourceReleaseThreads);
- return this;
- }
-
- /**
- * Get timeout for shutting down schedulers in dl manager, in milliseconds.
- * By default, it is 5 seconds.
- *
- * @return timeout for shutting down schedulers in dl manager, in miliseconds.
- */
- public int getSchedulerShutdownTimeoutMs() {
- return getInt(BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS, BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS_DEFAULT);
- }
-
- /**
- * Set timeout for shutting down schedulers in dl manager, in milliseconds.
- *
- * @param timeoutMs
- * timeout for shutting down schedulers in dl manager, in milliseconds.
- * @return dl configuration.
- * @see #getSchedulerShutdownTimeoutMs()
- */
- public DistributedLogConfiguration setSchedulerShutdownTimeoutMs(int timeoutMs) {
- setProperty(BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS, timeoutMs);
- return this;
- }
-
- /**
- * Whether to use daemon thread for DL threads.
- * By default it is false.
- *
- * @return true if use daemon threads, otherwise false.
- */
- public boolean getUseDaemonThread() {
- return getBoolean(BKDL_USE_DAEMON_THREAD, BKDL_USE_DAEMON_THREAD_DEFAULT);
- }
-
- /**
- * Set whether to use daemon thread for DL threads.
- *
- * @param daemon
- * whether to use daemon thread for DL threads.
- * @return distributedlog configuration
- * @see #getUseDaemonThread()
- */
- public DistributedLogConfiguration setUseDaemonThread(boolean daemon) {
- setProperty(BKDL_USE_DAEMON_THREAD, daemon);
- return this;
- }
-
- //
- // Metadata Settings
- //
-
- /**
- * Get DL ledger metadata output layout version.
- *
- * @return layout version
- * @see com.twitter.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion
- */
- public int getDLLedgerMetadataLayoutVersion() {
- return this.getInt(BKDL_LEDGER_METADATA_LAYOUT_VERSION,
- getInt(BKDL_LEDGER_METADATA_LAYOUT_VERSION_OLD,
- BKDL_LEDGER_METADATA_LAYOUT_VERSION_DEFAULT));
- }
-
- /**
- * Set DL ledger metadata output layout version.
- *
- * @param layoutVersion layout version
- * @return distributed log configuration
- * @throws IllegalArgumentException if setting an unknown layout version.
- * @see #getDLLedgerMetadataLayoutVersion()
- */
- public DistributedLogConfiguration setDLLedgerMetadataLayoutVersion(int layoutVersion)
- throws IllegalArgumentException {
- if ((layoutVersion <= 0) ||
- (layoutVersion > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION)) {
- // Incorrect version specified
- throw new IllegalArgumentException("Incorrect value for ledger metadata layout version");
- }
- setProperty(BKDL_LEDGER_METADATA_LAYOUT_VERSION, layoutVersion);
- return this;
- }
-
- /**
- * Get the setting for whether we should enforce the min ledger metadata version check.
- * By default it is false.
- *
- * @return whether we should enforce the min ledger metadata version check
- * @see com.twitter.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion
- */
- public boolean getDLLedgerMetadataSkipMinVersionCheck() {
- return this.getBoolean(BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK,
- BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK_DEFAULT);
- }
-
- /**
- * Set if we should skip the enforcement of min ledger metadata version.
- * <p>NOTE: please be aware the side effects of skipping min ledger metadata
- * version checking.
- *
- * @param skipMinVersionCheck whether we should enforce the min ledger metadata version check
- * @return distributed log configuration
- * @see #getDLLedgerMetadataSkipMinVersionCheck()
- */
- public DistributedLogConfiguration setDLLedgerMetadataSkipMinVersionCheck(boolean skipMinVersionCheck) throws IllegalArgumentException {
- setProperty(BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK, skipMinVersionCheck);
- return this;
- }
-
- /**
- * Get the value at which ledger sequence number should start for streams that are being
- * upgraded and did not have ledger sequence number to start with or for newly created
- * streams. By default, it is 1.
- * <p>In most of the cases this value should not be changed. It is useful for backfilling
- * in the case of migrating log segments whose metadata don't have log segment sequence number.
- *
- * @return first ledger sequence number
- */
- public long getFirstLogSegmentSequenceNumber() {
- return this.getLong(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER,
- getLong(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_OLD,
- BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_DEFAULT));
- }
-
- /**
- * Set the value at which ledger sequence number should start for streams that are being
- * upgraded and did not have ledger sequence number to start with or for newly created
- * streams
- *
- * @param firstLogSegmentSequenceNumber first ledger sequence number
- * @return distributed log configuration
- * @see #getFirstLogSegmentSequenceNumber()
- */
- public DistributedLogConfiguration setFirstLogSegmentSequenceNumber(long firstLogSegmentSequenceNumber)
- throws IllegalArgumentException {
- if (firstLogSegmentSequenceNumber <= 0) {
- // Incorrect ledger sequence number specified
- throw new IllegalArgumentException("Incorrect value for ledger sequence number");
- }
- setProperty(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER, firstLogSegmentSequenceNumber);
- return this;
- }
-
- /**
- * Whether log segment sequence number validation is enabled?
- *
- * @return true if the log segment sequence number validation is enabled, otherwise false.
- */
- public boolean isLogSegmentSequenceNumberValidationEnabled() {
- return this.getBoolean(BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED,
- BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED_DEFAULT);
- }
-
- /**
- * Whether log segment sequence number validation is enabled?
- *
- * @return true if the log segment sequence number validation is enabled, otherwise false.
- */
- public DistributedLogConfiguration setLogSegmentSequenceNumberValidationEnabled(boolean enabled) {
- setProperty(BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED, enabled);
- return this;
- }
-
- /**
- * Whether we should publish record counts in the log records and metadata.
- * <p>By default it is true. This is a legacy setting for log segment version 1. It
- * should be considered removed.
- *
- * @return if record counts should be persisted
- */
- public boolean getEnableRecordCounts() {
- return getBoolean(BKDL_ENABLE_RECORD_COUNTS, BKDL_ENABLE_RECORD_COUNTS_DEFAULT);
- }
-
- /**
- * Set if we should publish record counts in the log records and metadata.
- *
- * @param enableRecordCounts enable record counts
- * @return distributed log configuration
- * @see #getEnableRecordCounts()
- */
- public DistributedLogConfiguration setEnableRecordCounts(boolean enableRecordCounts) {
- setProperty(BKDL_ENABLE_RECORD_COUNTS, enableRecordCounts);
- return this;
- }
-
- /**
- * Whether sanity check txn id on starting log segments.
- * <p>If it is enabled, DL writer would throw
- * {@link com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException}
- * when it received a smaller transaction id than current maximum transaction id.
- *
- * @return true if should check txn id with max txn id, otherwise false.
- */
- @Deprecated
- public boolean getSanityCheckTxnID() {
- return getBoolean(BKDL_MAXID_SANITYCHECK, BKDL_MAXID_SANITYCHECK_DEFAULT);
- }
-
- /**
- * Enable/Disable sanity check txn id.
- *
- * @param enabled
- * enable/disable sanity check txn id.
- * @return configuration.
- * @see #getSanityCheckTxnID()
- */
- @Deprecated
- public DistributedLogConfiguration setSanityCheckTxnID(boolean enabled) {
- setProperty(BKDL_MAXID_SANITYCHECK, enabled);
- return this;
- }
-
- /**
- * Whether encode region id in log segment metadata.
- * <p>In global DL use case, encoding region id in log segment medata would
- * help understanding what region that a log segment is created. The region
- * id field in log segment metadata would help for moniotring and troubleshooting.
- *
- * @return whether to encode region id in log segment metadata.
- */
- public boolean getEncodeRegionIDInLogSegmentMetadata() {
- return getBoolean(BKDL_ENCODE_REGION_ID_IN_VERSION, BKDL_ENCODE_REGION_ID_IN_VERSION_DEFAULT);
- }
-
- /**
- * Enable/Disable encoding region id in log segment metadata.
- *
- * @param enabled
- * flag to enable/disable encoding region id in log segment metadata.
- * @return configuration instance.
- * @see #getEncodeRegionIDInLogSegmentMetadata()
- */
- public DistributedLogConfiguration setEncodeRegionIDInLogSegmentMetadata(boolean enabled) {
- setProperty(BKDL_ENCODE_REGION_ID_IN_VERSION, enabled);
- return this;
- }
-
- /**
- * Get log segment name version.
- * <p>
- * <ul>
- * <li>version 0: inprogress_(start_txid) |
- * logrecs_(start_txid)_(end_txid)</li>
- * <li>version 1: inprogress_(logsegment_sequence_number) |
- * logrecs_(logsegment_sequence_number)</li>
- * </ul>
- * By default it is 1.
- *
- * @return log segment name verison.
- */
- public int getLogSegmentNameVersion() {
- return getInt(BKDL_LOGSEGMENT_NAME_VERSION, BKDL_LOGSEGMENT_NAME_VERSION_DEFAULT);
- }
-
- /**
- * Set log segment name version.
- *
- * @param version
- * log segment name version.
- * @return configuration object.
- * @see #getLogSegmentNameVersion()
- */
- public DistributedLogConfiguration setLogSegmentNameVersion(int version) {
- setProperty(BKDL_LOGSEGMENT_NAME_VERSION, version);
- return this;
- }
-
- /**
- * Get name of the unpartitioned stream.
- * <p>It is a legacy setting. consider removing it in future.
- *
- * @return unpartitioned stream
- */
- public String getUnpartitionedStreamName() {
- return getString(BKDL_UNPARTITIONED_STREAM_NAME, BKDL_UNPARTITIONED_STREAM_NAME_DEFAULT);
- }
-
- /**
- * Set name of the unpartitioned stream
- *
- * @param streamName name of the unpartitioned stream
- * @return distributedlog configuration
- * @see #getUnpartitionedStreamName()
- */
- public DistributedLogConfiguration setUnpartitionedStreamName(String streamName) {
- setProperty(BKDL_UNPARTITIONED_STREAM_NAME, streamName);
- return this;
- }
-
- //
- // LogSegment Cache Settings
- //
-
- /**
- * Get the log segment cache entry TTL in milliseconds.
- *
- * @return log segment cache ttl in milliseconds.
- */
- public long getLogSegmentCacheTTLMs() {
- return getLong(BKDL_LOGSEGMENT_CACHE_TTL_MS, BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT);
- }
-
- /**
- * Set the log segment cache entry TTL in milliseconds.
- *
- * @param ttlMs TTL in milliseconds
- * @return distributedlog configuration
- */
- public DistributedLogConfiguration setLogSegmentCacheTTLMs(long ttlMs) {
- setProperty(BKDL_LOGSEGMENT_CACHE_TTL_MS, ttlMs);
- return this;
- }
-
- /**
- * Get the maximum size of the log segment cache.
- *
- * @return maximum size of the log segment cache.
- */
- public long getLogSegmentCacheMaxSize() {
- return getLong(BKDL_LOGSEGMENT_CACHE_MAX_SIZE, BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT);
- }
-
- /**
- * Set the maximum size of the log segment cache.
- *
- * @param maxSize maximum size of the log segment cache.
- * @return distributedlog configuration
- */
- public DistributedLogConfiguration setLogSegmentCacheMaxSize(long maxSize) {
- setProperty(BKDL_LOGSEGMENT_CACHE_MAX_SIZE, maxSize);
- return this;
- }
-
- /**
- * Is log segment cache enabled?
- *
- * @return true if log segment cache is enabled; otherwise false
- */
- public boolean isLogSegmentCacheEnabled() {
- return getBoolean(BKDL_LOGSEGMENT_CACHE_ENABLED, BKDL_LOGSEGMENT_CACHE_ENABLED_DEFAULT);
- }
-
- /**
- * Enable/disable log segment cache.
- *
- * @return distributedlog configuration
- */
- public DistributedLogConfiguration setLogSegmentCacheEnabled(boolean enabled) {
- setProperty(BKDL_LOGSEGMENT_CACHE_ENABLED, enabled);
- return this;
- }
-
- //
- // DL Writer General Settings
- //
-
- /**
- * Whether to create stream if not exists. By default it is true.
- *
- * @return true if it is abled to create stream if not exists.
- */
- public boolean getCreateStreamIfNotExists() {
- return getBoolean(BKDL_CREATE_STREAM_IF_NOT_EXISTS,
- BKDL_CREATE_STREAM_IF_NOT_EXISTS_DEFAULT);
- }
-
- /**
- * Enable/Disable creating stream if not exists.
- *
- * @param enabled
- * enable/disable sanity check txn id.
- * @return distributed log configuration.
- * @see #getCreateStreamIfNotExists()
- */
- public DistributedLogConfiguration setCreateStreamIfNotExists(boolean enabled) {
- setProperty(BKDL_CREATE_STREAM_IF_NOT_EXISTS, enabled);
- return this;
- }
-
- /**
- * Get Log Flush timeout in seconds.
- * <p>This is a setting used by DL writer on flushing data. It is typically used
- * by synchronous writer and log segment writer. By default it is 30 seconds.
- *
- * @return log flush timeout in seconds.
- */
- // @Deprecated
- public int getLogFlushTimeoutSeconds() {
- return this.getInt(BKDL_LOG_FLUSH_TIMEOUT, BKDL_LOG_FLUSH_TIMEOUT_DEFAULT);
- }
-
- /**
- * Set Log Flush Timeout in seconds.
- *
- * @param logFlushTimeoutSeconds log flush timeout.
- * @return distributed log configuration
- * @see #getLogFlushTimeoutSeconds()
- */
- public DistributedLogConfiguration setLogFlushTimeoutSeconds(int logFlushTimeoutSeconds) {
- setProperty(BKDL_LOG_FLUSH_TIMEOUT, logFlushTimeoutSeconds);
- return this;
- }
-
- /**
- * The compression type to use while sending data to bookkeeper.
- *
- * @return compression type to use
- * @see com.twitter.distributedlog.io.CompressionCodec
- */
- public String getCompressionType() {
- return getString(BKDL_COMPRESSION_TYPE, BKDL_COMPRESSION_TYPE_DEFAULT);
- }
-
- /**
- * Set the compression type to use while sending data to bookkeeper.
- *
- * @param compressionType compression type
- * @return distributedlog configuration
- * @see #getCompressionType()
- */
- public DistributedLogConfiguration setCompressionType(String compressionType) {
- Preconditions.checkArgument(null != compressionType && !compressionType.isEmpty());
- setProperty(BKDL_COMPRESSION_TYPE, compressionType);
- return this;
- }
-
- /**
- * Whether to fail immediately if the stream is not ready rather than queueing the request.
- * <p>If it is enabled, it would fail the write request immediately if the stream isn't ready.
- * Consider turning it on for the use cases that could retry writing to other streams
- * (aka non-strict ordering guarantee). It would result fast failure hence the client would
- * retry immediately.
- *
- * @return true if should fail fast. otherwise, false.
- */
- public boolean getFailFastOnStreamNotReady() {
- return getBoolean(BKDL_FAILFAST_ON_STREAM_NOT_READY,
- BKDL_FAILFAST_ON_STREAM_NOT_READY_DEFAULT);
- }
-
- /**
- * Set the failfast on stream not ready flag.
- *
- * @param failFastOnStreamNotReady
- * set failfast flag
- * @return dl configuration.
- * @see #getFailFastOnStreamNotReady()
- */
- public DistributedLogConfiguration setFailFastOnStreamNotReady(boolean failFastOnStreamNotReady) {
- setProperty(BKDL_FAILFAST_ON_STREAM_NOT_READY, failFastOnStreamNotReady);
- return this;
- }
-
- /**
- * If this option is set, the log writer won't reset the segment writer if an error
- * is encountered.
- *
- * @return true if we should disable automatic rolling
- */
- public boolean getDisableRollingOnLogSegmentError() {
- return getBoolean(BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR,
- BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR_DEFAULT);
- }
-
- /**
- * Set the roll on segment error flag.
- *
- * @param disableRollingOnLogSegmentError
- * set roll on error flag
- * @return dl configuration.
- * @see #getDisableRollingOnLogSegmentError()
- */
- public DistributedLogConfiguration setDisableRollingOnLogSegmentError(boolean disableRollingOnLogSegmentError) {
- setProperty(BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR, disableRollingOnLogSegmentError);
- return this;
- }
-
- //
- // DL Durability Settings
- //
-
- /**
- * Check whether the durable write is enabled.
- * <p>It is enabled by default.
- *
- * @return true if durable write is enabled. otherwise, false.
- */
- public boolean isDurableWriteEnabled() {
- return this.getBoolean(BKDL_IS_DURABLE_WRITE_ENABLED, BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT);
- }
-
- /**
- * Enable/Disable durable writes in writers.
- *
- * @param enabled
- * flag to enable/disable durable writes in writers.
- * @return distributedlog configuration
- */
- public DistributedLogConfiguration setDurableWriteEnabled(boolean enabled) {
- setProperty(BKDL_IS_DURABLE_WRITE_ENABLED, enabled);
- return this;
- }
-
- //
- // DL Writer Transmit Settings
- //
-
- /**
- * Get output buffer size for DL writers, in bytes.
- * <p>Large buffer will result in higher compression ratio and
- * it would use the bandwidth more efficiently and improve throughput.
- * Set it to 0 would ask DL writers to transmit the data immediately,
- * which it could achieve low latency.
- * <p>The default value is 1KB.
- *
- * @return buffer size in byes.
- */
- public int getOutputBufferSize() {
- return this.getInt(BKDL_OUTPUT_BUFFER_SIZE,
- getInt(BKDL_OUTPUT_BUFFER_SIZE_OLD, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT));
- }
-
- /**
- * Set output buffer size for DL writers, in bytes.
- *
- * @param opBufferSize output buffer size.
- * @return distributed log configuration
- * @see #getOutputBufferSize()
- */
- public DistributedLogConfiguration setOutputBufferSize(int opBufferSize) {
- setProperty(BKDL_OUTPUT_BUFFER_SIZE, opBufferSize);
- return this;
- }
-
- /**
- * Get Periodic Log Flush Frequency in milliseconds.
- * <p>If the setting is set with a positive value, the data in output buffer
- * will be flushed in this provided interval. The default value is 0.
- *
- * @return periodic flush frequency in milliseconds.
- * @see #getOutputBufferSize()
- */
- public int getPeriodicFlushFrequencyMilliSeconds() {
- return this.getInt(BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
- BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT);
- }
-
- /**
- * Set Periodic Log Flush Frequency in milliseconds.
- *
- * @param flushFrequencyMs periodic flush frequency in milliseconds.
- * @return distributed log configuration
- * @see #getPeriodicFlushFrequencyMilliSeconds()
- */
- public DistributedLogConfiguration setPeriodicFlushFrequencyMilliSeconds(int flushFrequencyMs) {
- setProperty(BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS, flushFrequencyMs);
- return this;
- }
-
- /**
- * Is immediate flush enabled.
- * <p>If it is enabled, it would flush control record immediately after adding
- * data completed. The default value is false.
- *
- * @return whether immediate flush is enabled
- */
- public boolean getImmediateFlushEnabled() {
- return getBoolean(BKDL_ENABLE_IMMEDIATE_FLUSH, BKDL_ENABLE_IMMEDIATE_FLUSH_DEFAULT);
- }
-
- /**
- * Enable/Disable immediate flush
- *
- * @param enabled
- * flag to enable/disable immediate flush.
- * @return configuration instance.
- * @see #getImmediateFlushEnabled()
- */
- public DistributedLogConfiguration setImmediateFlushEnabled(boolean enabled) {
- setProperty(BKDL_ENABLE_IMMEDIATE_FLUSH, enabled);
- return this;
- }
-
- /**
- * Get minimum delay between immediate flushes in milliseconds.
- * <p>This setting only takes effects when {@link #getImmediateFlushEnabled()}
- * is enabled. It torelants the bursty of traffic when immediate flush is enabled,
- * which prevents sending too many control records to the bookkeeper.
- *
- * @return minimum delay between immediate flushes in milliseconds
- * @see #getImmediateFlushEnabled()
- */
- public int getMinDelayBetweenImmediateFlushMs() {
- return this.getInt(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS, BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT);
- }
-
- /**
- * Set minimum delay between immediate flushes in milliseconds
- *
- * @param minDelayMs minimum delay between immediate flushes in milliseconds.
- * @return distributed log configuration
- * @see #getMinDelayBetweenImmediateFlushMs()
- */
- public DistributedLogConfiguration setMinDelayBetweenImmediateFlushMs(int minDelayMs) {
- setProperty(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS, minDelayMs);
- return this;
- }
-
- /**
- * Get Periodic Keep Alive Frequency in milliseconds.
- * <p>If the setting is set with a positive value, it would periodically write a control record
- * to keep the stream active. The default value is 0.
- *
- * @return periodic keep alive frequency in milliseconds.
- */
- public int getPeriodicKeepAliveMilliSeconds() {
- return this.getInt(BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS, BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS_DEFAULT);
- }
-
- /**
- * Set Periodic Keep Alive Frequency in milliseconds.
- *
- * @param keepAliveMs keep alive frequency in milliseconds.
- * @return distributedlog configuration
- * @see #getPeriodicKeepAliveMilliSeconds()
- */
- public DistributedLogConfiguration setPeriodicKeepAliveMilliSeconds(int keepAliveMs) {
- setProperty(BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS, keepAliveMs);
- return this;
- }
-
- //
- // DL Retention/Truncation Settings
- //
-
- /**
- * Get log segment retention period in hours.
- * The default value is 3 days.
- *
- * @return log segment retention period in hours
- */
- public int getRetentionPeriodHours() {
- return this.getInt(BKDL_RETENTION_PERIOD_IN_HOURS,
- getInt(BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
- BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT));
- }
-
- /**
- * Set log segment retention period in hours.
- *
- * @param retentionHours retention period in hours.
- * @return distributed log configuration
- */
- public DistributedLogConfiguration setRetentionPeriodHours(int retentionHours) {
- setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, retentionHours);
- return this;
- }
-
- /**
- * Is truncation managed explicitly by the application.
- * <p>If this is set then time based retention is only a hint to perform
- * deferred cleanup. However we never remove a segment that has not been
- * already marked truncated.
- * <p>It is disabled by default.
- *
- * @return whether truncation managed explicitly by the application
- * @see com.twitter.distributedlog.LogSegmentMetadata.TruncationStatus
- */
- public boolean getExplicitTruncationByApplication() {
- return getBoolean(BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION,
- BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION_DEFAULT);
- }
-
- /**
- * Enable/Disable whether truncation is managed explicitly by the application.
- *
- * @param enabled
- * flag to enable/disable whether truncation is managed explicitly by the application.
- * @return configuration instance.
- */
- public DistributedLogConfiguration setExplicitTruncationByApplication(boolean enabled) {
- setProperty(BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION, enabled);
- return this;
- }
-
- //
- // Log Segment Rolling Settings
- //
-
- /**
- * Get log segment rolling interval in minutes.
- * <p>If the setting is set to a positive value, DL writer will roll log segments
- * based on time. Otherwise, it will roll log segments based on size.
- * <p>The default value is 2 hours.
- *
- * @return log segment rolling interval in minutes
-
<TRUNCATED>
[13/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKVersionedSetOp.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKVersionedSetOp.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKVersionedSetOp.java
deleted file mode 100644
index 5b788e2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKVersionedSetOp.java
+++ /dev/null
@@ -1,71 +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.zk;
-
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-
-import javax.annotation.Nullable;
-
-/**
- * ZooKeeper Operation that plays with {@link org.apache.bookkeeper.versioning.Version}
- */
-public class ZKVersionedSetOp extends ZKOp {
-
- private final OpListener<Version> listener;
-
- public ZKVersionedSetOp(Op op,
- @Nullable OpListener<Version> opListener) {
- super(op);
- this.listener = opListener;
- }
-
- @Override
- protected void commitOpResult(OpResult opResult) {
- assert(opResult instanceof OpResult.SetDataResult);
- OpResult.SetDataResult setDataResult = (OpResult.SetDataResult) opResult;
- if (null != listener) {
- listener.onCommit(new ZkVersion(setDataResult.getStat().getVersion()));
- }
- }
-
- @Override
- protected void abortOpResult(Throwable t,
- @Nullable OpResult opResult) {
- Throwable cause;
- if (null == opResult) {
- cause = t;
- } else {
- assert (opResult instanceof OpResult.ErrorResult);
- OpResult.ErrorResult errorResult = (OpResult.ErrorResult) opResult;
- if (KeeperException.Code.OK.intValue() == errorResult.getErr()) {
- cause = t;
- } else {
- cause = KeeperException.create(KeeperException.Code.get(errorResult.getErr()));
- }
- }
- if (null != listener) {
- listener.onAbort(cause);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKWatcherManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKWatcherManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKWatcherManager.java
deleted file mode 100644
index 8ef33ea..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKWatcherManager.java
+++ /dev/null
@@ -1,239 +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.zk;
-
-import com.twitter.distributedlog.ZooKeeperClient;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Watcher Manager to manage watchers.
- * <h3>Metrics</h3>
- * <ul>
- * <li> `total_watches`: total number of watches that managed by this watcher manager.
- * <li> `num_child_watches`: number of paths that are watched for children changes by this watcher manager.
- * </ul>
- */
-public class ZKWatcherManager implements Watcher {
-
- static final Logger logger = LoggerFactory.getLogger(ZKWatcherManager.class);
-
- public static Builder newBuilder() {
- return new Builder();
- }
-
- public static class Builder {
-
- private String _name;
- private StatsLogger _statsLogger;
- private ZooKeeperClient _zkc;
-
- public Builder name(String name) {
- this._name = name;
- return this;
- }
-
- public Builder zkc(ZooKeeperClient zkc) {
- this._zkc = zkc;
- return this;
- }
-
- public Builder statsLogger(StatsLogger statsLogger) {
- this._statsLogger = statsLogger;
- return this;
- }
-
- public ZKWatcherManager build() {
- return new ZKWatcherManager(_name, _zkc, _statsLogger);
- }
- }
-
- private final String name;
- private final ZooKeeperClient zkc;
- private final StatsLogger statsLogger;
- // Gauges and their labels
- private final Gauge<Number> totalWatchesGauge;
- private static final String totalWatchesGauageLabel = "total_watches";
- private final Gauge<Number> numChildWatchesGauge;
- private static final String numChildWatchesGauageLabel = "num_child_watches";
-
- protected final ConcurrentMap<String, Set<Watcher>> childWatches;
- protected final AtomicInteger allWatchesGauge;
-
- private ZKWatcherManager(String name,
- ZooKeeperClient zkc,
- StatsLogger statsLogger) {
- this.name = name;
- this.zkc = zkc;
- this.statsLogger = statsLogger;
-
- // watches
- this.childWatches = new ConcurrentHashMap<String, Set<Watcher>>();
- this.allWatchesGauge = new AtomicInteger(0);
-
- // stats
- totalWatchesGauge = new Gauge<Number>() {
- @Override
- public Number getDefaultValue() {
- return 0;
- }
-
- @Override
- public Number getSample() {
- return allWatchesGauge.get();
- }
- };
- this.statsLogger.registerGauge(totalWatchesGauageLabel, totalWatchesGauge);
-
- numChildWatchesGauge = new Gauge<Number>() {
- @Override
- public Number getDefaultValue() {
- return 0;
- }
-
- @Override
- public Number getSample() {
- return childWatches.size();
- }
- };
-
- this.statsLogger.registerGauge(numChildWatchesGauageLabel, numChildWatchesGauge);
- }
-
- public Watcher registerChildWatcher(String path, Watcher watcher) {
- Set<Watcher> watchers = childWatches.get(path);
- if (null == watchers) {
- Set<Watcher> newWatchers = new HashSet<Watcher>();
- Set<Watcher> oldWatchers = childWatches.putIfAbsent(path, newWatchers);
- watchers = (null == oldWatchers) ? newWatchers : oldWatchers;
- }
- synchronized (watchers) {
- if (childWatches.get(path) == watchers) {
- if (watchers.add(watcher)) {
- allWatchesGauge.incrementAndGet();
- }
- } else {
- logger.warn("Watcher set for path {} has been changed while registering child watcher {}.",
- path, watcher);
- }
- }
- return this;
- }
-
- public void unregisterChildWatcher(String path, Watcher watcher, boolean removeFromServer) {
- Set<Watcher> watchers = childWatches.get(path);
- if (null == watchers) {
- logger.warn("No watchers found on path {} while unregistering child watcher {}.",
- path, watcher);
- return;
- }
- synchronized (watchers) {
- if (watchers.remove(watcher)) {
- allWatchesGauge.decrementAndGet();
- } else {
- logger.warn("Remove a non-registered child watcher {} from path {}", watcher, path);
- }
- if (watchers.isEmpty()) {
- // best-efforts to remove watches
- try {
- if (null != zkc && removeFromServer) {
- zkc.get().removeWatches(path, this, WatcherType.Children, true, new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx) {
- if (KeeperException.Code.OK.intValue() == rc) {
- logger.debug("Successfully removed children watches from {}", path);
- } else {
- logger.debug("Encountered exception on removing children watches from {}",
- path, KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
- }, null);
- }
- } catch (InterruptedException e) {
- logger.debug("Encountered exception on removing watches from {}", path, e);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- logger.debug("Encountered exception on removing watches from {}", path, e);
- }
- childWatches.remove(path, watchers);
- }
- }
- }
-
- public void unregisterGauges() {
- this.statsLogger.unregisterGauge(totalWatchesGauageLabel, totalWatchesGauge);
- this.statsLogger.unregisterGauge(numChildWatchesGauageLabel, numChildWatchesGauge);
- }
-
- @Override
- public void process(WatchedEvent event) {
- switch (event.getType()) {
- case None:
- handleKeeperStateEvent(event);
- break;
- case NodeChildrenChanged:
- handleChildWatchEvent(event);
- break;
- default:
- break;
- }
- }
-
- private void handleKeeperStateEvent(WatchedEvent event) {
- Set<Watcher> savedAllWatches = new HashSet<Watcher>(allWatchesGauge.get());
- for (Set<Watcher> watcherSet : childWatches.values()) {
- synchronized (watcherSet) {
- savedAllWatches.addAll(watcherSet);
- }
- }
- for (Watcher watcher : savedAllWatches) {
- watcher.process(event);
- }
- }
-
- private void handleChildWatchEvent(WatchedEvent event) {
- String path = event.getPath();
- if (null == path) {
- logger.warn("Received zookeeper watch event with null path : {}", event);
- return;
- }
- Set<Watcher> watchers = childWatches.get(path);
- if (null == watchers) {
- return;
- }
- Set<Watcher> watchersToFire;
- synchronized (watchers) {
- watchersToFire = new HashSet<Watcher>(watchers.size());
- watchersToFire.addAll(watchers);
- }
- for (Watcher watcher : watchersToFire) {
- watcher.process(event);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
new file mode 100644
index 0000000..1d96f0e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
@@ -0,0 +1,198 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AppendOnlyStreamReader extends InputStream {
+ static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamReader.class);
+
+ private LogRecordWithInputStream currentLogRecord = null;
+ private final DistributedLogManager dlm;
+ private LogReader reader;
+ private long currentPosition;
+ private static final int SKIP_BUFFER_SIZE = 512;
+
+ // Cache the input stream for a log record.
+ private static class LogRecordWithInputStream {
+ private final InputStream payloadStream;
+ private final LogRecordWithDLSN logRecord;
+
+ LogRecordWithInputStream(LogRecordWithDLSN logRecord) {
+ Preconditions.checkNotNull(logRecord);
+
+ LOG.debug("Got record dlsn = {}, txid = {}, len = {}",
+ new Object[] {logRecord.getDlsn(), logRecord.getTransactionId(), logRecord.getPayload().length});
+
+ this.logRecord = logRecord;
+ this.payloadStream = logRecord.getPayLoadInputStream();
+ }
+
+ InputStream getPayLoadInputStream() {
+ return payloadStream;
+ }
+
+ LogRecordWithDLSN getLogRecord() {
+ return logRecord;
+ }
+
+ // The last txid of the log record is the position of the next byte in the stream.
+ // Subtract length to get starting offset.
+ long getOffset() {
+ return logRecord.getTransactionId() - logRecord.getPayload().length;
+ }
+ }
+
+ /**
+ * Construct ledger input stream
+ *
+ * @param dlm the Distributed Log Manager to access the stream
+ */
+ AppendOnlyStreamReader(DistributedLogManager dlm)
+ throws IOException {
+ this.dlm = dlm;
+ reader = dlm.getInputStream(0);
+ currentPosition = 0;
+ }
+
+ /**
+ * Get input stream representing next entry in the
+ * ledger.
+ *
+ * @return input stream, or null if no more entries
+ */
+ private LogRecordWithInputStream nextLogRecord() throws IOException {
+ return nextLogRecord(reader);
+ }
+
+ private static LogRecordWithInputStream nextLogRecord(LogReader reader) throws IOException {
+ LogRecordWithDLSN record = reader.readNext(false);
+
+ if (null != record) {
+ return new LogRecordWithInputStream(record);
+ } else {
+ record = reader.readNext(false);
+ if (null != record) {
+ return new LogRecordWithInputStream(record);
+ } else {
+ LOG.debug("No record");
+ return null;
+ }
+ }
+ }
+
+ @Override
+ public int read() throws IOException {
+ byte[] b = new byte[1];
+ if (read(b, 0, 1) != 1) {
+ return -1;
+ } else {
+ return b[0];
+ }
+ }
+
+ @Override
+ public int read(byte[] b, int off, int len) throws IOException {
+ int read = 0;
+ if (currentLogRecord == null) {
+ currentLogRecord = nextLogRecord();
+ if (currentLogRecord == null) {
+ return read;
+ }
+ }
+
+ while (read < len) {
+ int thisread = currentLogRecord.getPayLoadInputStream().read(b, off + read, (len - read));
+ if (thisread == -1) {
+ currentLogRecord = nextLogRecord();
+ if (currentLogRecord == null) {
+ return read;
+ }
+ } else {
+ LOG.debug("Offset saved = {}, persisted = {}",
+ currentPosition, currentLogRecord.getLogRecord().getTransactionId());
+ currentPosition += thisread;
+ read += thisread;
+ }
+ }
+ return read;
+ }
+
+ /**
+ * Position the reader at the given offset. If we fail to skip to the desired position
+ * and don't hit end of stream, return false.
+ *
+ * @throws org.apache.distributedlog.exceptions.EndOfStreamException if we attempt to
+ * skip past the end of the stream.
+ */
+ public boolean skipTo(long position) throws IOException {
+
+ // No need to skip anywhere.
+ if (position == position()) {
+ return true;
+ }
+
+ LogReader skipReader = dlm.getInputStream(position);
+ LogRecordWithInputStream logRecord = null;
+ try {
+ logRecord = nextLogRecord(skipReader);
+ } catch (IOException ex) {
+ skipReader.close();
+ throw ex;
+ }
+
+ if (null == logRecord) {
+ return false;
+ }
+
+ // We may end up with a reader positioned *before* the requested position if
+ // we're near the tail and the writer is still active, or if the desired position
+ // is not at a log record payload boundary.
+ // Transaction ID gives us the starting position of the log record. Read ahead
+ // if necessary.
+ currentPosition = logRecord.getOffset();
+ currentLogRecord = logRecord;
+ LogReader oldReader = reader;
+ reader = skipReader;
+
+ // Close the oldreader after swapping AppendOnlyStreamReader state. Close may fail
+ // and we need to make sure it leaves AppendOnlyStreamReader in a consistent state.
+ oldReader.close();
+
+ byte[] skipBuffer = new byte[SKIP_BUFFER_SIZE];
+ while (currentPosition < position) {
+ long bytesToRead = Math.min(position - currentPosition, SKIP_BUFFER_SIZE);
+ long bytesRead = read(skipBuffer, 0, (int)bytesToRead);
+ if (bytesRead < bytesToRead) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ public long position() {
+ return currentPosition;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
new file mode 100644
index 0000000..8278c68
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
@@ -0,0 +1,107 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.util.FutureUtils;
+import com.twitter.util.Await;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AppendOnlyStreamWriter implements Closeable {
+ static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamWriter.class);
+
+ // Use a 1-length array to satisfy Java's inner class reference rules. Use primitive
+ // type because synchronized block is needed anyway.
+ final long[] syncPos = new long[1];
+ BKAsyncLogWriter logWriter;
+ long requestPos = 0;
+
+ public AppendOnlyStreamWriter(BKAsyncLogWriter logWriter, long pos) {
+ LOG.debug("initialize at position {}", pos);
+ this.logWriter = logWriter;
+ this.syncPos[0] = pos;
+ this.requestPos = pos;
+ }
+
+ public Future<DLSN> write(byte[] data) {
+ requestPos += data.length;
+ Future<DLSN> writeResult = logWriter.write(new LogRecord(requestPos, data));
+ return writeResult.addEventListener(new WriteCompleteListener(requestPos));
+ }
+
+ public void force(boolean metadata) throws IOException {
+ long pos = 0;
+ try {
+ pos = Await.result(logWriter.flushAndCommit());
+ } catch (IOException ioe) {
+ throw ioe;
+ } catch (Exception ex) {
+ LOG.error("unexpected exception in AppendOnlyStreamWriter.force ", ex);
+ throw new UnexpectedException("unexpected exception in AppendOnlyStreamWriter.force", ex);
+ }
+ synchronized (syncPos) {
+ syncPos[0] = pos;
+ }
+ }
+
+ public long position() {
+ synchronized (syncPos) {
+ return syncPos[0];
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ logWriter.closeAndComplete();
+ }
+
+ public void markEndOfStream() throws IOException {
+ try {
+ Await.result(logWriter.markEndOfStream());
+ } catch (IOException ioe) {
+ throw ioe;
+ } catch (Exception ex) {
+ throw new UnexpectedException("Mark end of stream hit unexpected exception", ex);
+ }
+ }
+
+ class WriteCompleteListener implements FutureEventListener<DLSN> {
+ private final long position;
+ public WriteCompleteListener(long position) {
+ this.position = position;
+ }
+ @Override
+ public void onSuccess(DLSN response) {
+ synchronized (syncPos) {
+ if (position > syncPos[0]) {
+ syncPos[0] = position;
+ }
+ }
+ }
+ @Override
+ public void onFailure(Throwable cause) {
+ // Handled at the layer above
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java
new file mode 100644
index 0000000..e3ace05
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java
@@ -0,0 +1,69 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.io.AsyncCloseable;
+import com.twitter.util.Future;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public interface AsyncLogReader extends AsyncCloseable {
+
+ /**
+ * Get stream name that the reader reads from.
+ *
+ * @return stream name.
+ */
+ public String getStreamName();
+
+ /**
+ * Read the next record from the log stream
+ *
+ * @return A promise that when satisfied will contain the Log Record with its DLSN.
+ */
+ public Future<LogRecordWithDLSN> readNext();
+
+ /**
+ * Read next <i>numEntries</i> entries. The future is only satisfied with non-empty list
+ * of entries. It doesn't block until returning exact <i>numEntries</i>. It is a best effort
+ * call.
+ *
+ * @param numEntries
+ * num entries
+ * @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
+ */
+ public Future<List<LogRecordWithDLSN>> readBulk(int numEntries);
+
+ /**
+ * Read next <i>numEntries</i> entries in a given <i>waitTime</i>.
+ * <p>
+ * The future is satisfied when either reads <i>numEntries</i> entries or reaches <i>waitTime</i>.
+ * The only exception is if there isn't any new entries written within <i>waitTime</i>, it would
+ * wait until new entries are available.
+ *
+ * @param numEntries
+ * max entries to return
+ * @param waitTime
+ * maximum wait time if there are entries already for read
+ * @param timeUnit
+ * wait time unit
+ * @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
+ */
+ public Future<List<LogRecordWithDLSN>> readBulk(int numEntries, long waitTime, TimeUnit timeUnit);
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
new file mode 100644
index 0000000..53b393b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
@@ -0,0 +1,70 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.io.AsyncAbortable;
+import org.apache.distributedlog.io.AsyncCloseable;
+import com.twitter.util.Future;
+
+import java.io.Closeable;
+import java.util.List;
+
+public interface AsyncLogWriter extends AsyncCloseable, AsyncAbortable {
+
+ /**
+ * Get the last committed transaction id.
+ *
+ * @return last committed transaction id.
+ */
+ public long getLastTxId();
+
+ /**
+ * Write a log record to the stream.
+ *
+ * @param record single log record
+ * @return A Future which contains a DLSN if the record was successfully written
+ * or an exception if the write fails
+ */
+ public Future<DLSN> write(LogRecord record);
+
+ /**
+ * Write log records to the stream in bulk. Each future in the list represents the result of
+ * one write operation. The size of the result list is equal to the size of the input list.
+ * Buffers are written in order, and the list of result futures has the same order.
+ *
+ * @param record set of log records
+ * @return A Future which contains a list of Future DLSNs if the record was successfully written
+ * or an exception if the operation fails.
+ */
+ public Future<List<Future<DLSN>>> writeBulk(List<LogRecord> record);
+
+ /**
+ * Truncate the log until <i>dlsn</i>.
+ *
+ * @param dlsn
+ * dlsn to truncate until.
+ * @return A Future indicates whether the operation succeeds or not, or an exception
+ * if the truncation fails.
+ */
+ public Future<Boolean> truncate(DLSN dlsn);
+
+ /**
+ * Get the name of the stream this writer writes data to
+ */
+ public String getStreamName();
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
new file mode 100644
index 0000000..c12bd10
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
@@ -0,0 +1,32 @@
+/**
+ * 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 org.apache.distributedlog;
+
+public interface AsyncNotification {
+ /**
+ * Triggered when the background activity encounters an exception
+ *
+ * @param reason the exception that encountered.
+ */
+ void notifyOnError(Throwable reason);
+
+ /**
+ * Triggered when the background activity completes an operation
+ */
+ void notifyOnOperationComplete();
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
new file mode 100644
index 0000000..4a2ef30
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
@@ -0,0 +1,555 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.io.Abortable;
+import org.apache.distributedlog.io.Abortables;
+import org.apache.distributedlog.io.AsyncAbortable;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.PermitManager;
+import org.apache.distributedlog.util.Utils;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+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.util.List;
+import java.util.concurrent.TimeUnit;
+
+abstract class BKAbstractLogWriter implements Closeable, AsyncCloseable, Abortable, AsyncAbortable {
+ static final Logger LOG = LoggerFactory.getLogger(BKAbstractLogWriter.class);
+
+ protected final DistributedLogConfiguration conf;
+ private final DynamicDistributedLogConfiguration dynConf;
+ protected final BKDistributedLogManager bkDistributedLogManager;
+
+ // States
+ private Promise<Void> closePromise = null;
+ private volatile boolean forceRolling = false;
+ private boolean forceRecovery = false;
+
+ // Truncation Related
+ private Future<List<LogSegmentMetadata>> lastTruncationAttempt = null;
+ @VisibleForTesting
+ private Long minTimestampToKeepOverride = null;
+
+ // Log Segment Writers
+ protected BKLogSegmentWriter segmentWriter = null;
+ protected Future<BKLogSegmentWriter> segmentWriterFuture = null;
+ protected BKLogSegmentWriter allocatedSegmentWriter = null;
+ protected BKLogWriteHandler writeHandler = null;
+
+ BKAbstractLogWriter(DistributedLogConfiguration conf,
+ DynamicDistributedLogConfiguration dynConf,
+ BKDistributedLogManager bkdlm) {
+ this.conf = conf;
+ this.dynConf = dynConf;
+ this.bkDistributedLogManager = bkdlm;
+ LOG.debug("Initial retention period for {} : {}", bkdlm.getStreamName(),
+ TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS));
+ }
+
+ // manage write handler
+
+ synchronized protected BKLogWriteHandler getCachedWriteHandler() {
+ return writeHandler;
+ }
+
+ protected BKLogWriteHandler getWriteHandler() throws IOException {
+ BKLogWriteHandler writeHandler = createAndCacheWriteHandler();
+ writeHandler.checkMetadataException();
+ return writeHandler;
+ }
+
+ protected BKLogWriteHandler createAndCacheWriteHandler()
+ throws IOException {
+ synchronized (this) {
+ if (writeHandler != null) {
+ return writeHandler;
+ }
+ }
+ // This code path will be executed when the handler is not set or has been closed
+ // due to forceRecovery during testing
+ BKLogWriteHandler newHandler =
+ FutureUtils.result(bkDistributedLogManager.asyncCreateWriteHandler(false));
+ boolean success = false;
+ try {
+ synchronized (this) {
+ if (writeHandler == null) {
+ writeHandler = newHandler;
+ success = true;
+ }
+ return writeHandler;
+ }
+ } finally {
+ if (!success) {
+ newHandler.asyncAbort();
+ }
+ }
+ }
+
+ // manage log segment writers
+
+ protected synchronized BKLogSegmentWriter getCachedLogWriter() {
+ return segmentWriter;
+ }
+
+ protected synchronized Future<BKLogSegmentWriter> getCachedLogWriterFuture() {
+ return segmentWriterFuture;
+ }
+
+ protected synchronized void cacheLogWriter(BKLogSegmentWriter logWriter) {
+ this.segmentWriter = logWriter;
+ this.segmentWriterFuture = Future.value(logWriter);
+ }
+
+ protected synchronized BKLogSegmentWriter removeCachedLogWriter() {
+ try {
+ return segmentWriter;
+ } finally {
+ segmentWriter = null;
+ segmentWriterFuture = null;
+ }
+ }
+
+ protected synchronized BKLogSegmentWriter getAllocatedLogWriter() {
+ return allocatedSegmentWriter;
+ }
+
+ protected synchronized void cacheAllocatedLogWriter(BKLogSegmentWriter logWriter) {
+ this.allocatedSegmentWriter = logWriter;
+ }
+
+ protected synchronized BKLogSegmentWriter removeAllocatedLogWriter() {
+ try {
+ return allocatedSegmentWriter;
+ } finally {
+ allocatedSegmentWriter = null;
+ }
+ }
+
+ private Future<Void> asyncCloseAndComplete(boolean shouldThrow) {
+ BKLogSegmentWriter segmentWriter = getCachedLogWriter();
+ BKLogWriteHandler writeHandler = getCachedWriteHandler();
+ if (null != segmentWriter && null != writeHandler) {
+ cancelTruncation();
+ Promise<Void> completePromise = new Promise<Void>();
+ asyncCloseAndComplete(segmentWriter, writeHandler, completePromise, shouldThrow);
+ return completePromise;
+ } else {
+ return closeNoThrow();
+ }
+ }
+
+ private void asyncCloseAndComplete(final BKLogSegmentWriter segmentWriter,
+ final BKLogWriteHandler writeHandler,
+ final Promise<Void> completePromise,
+ final boolean shouldThrow) {
+ writeHandler.completeAndCloseLogSegment(segmentWriter)
+ .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+ @Override
+ public void onSuccess(LogSegmentMetadata segment) {
+ removeCachedLogWriter();
+ complete(null);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ LOG.error("Completing Log segments encountered exception", cause);
+ complete(cause);
+ }
+
+ private void complete(final Throwable cause) {
+ closeNoThrow().ensure(new AbstractFunction0<BoxedUnit>() {
+ @Override
+ public BoxedUnit apply() {
+ if (null != cause && shouldThrow) {
+ FutureUtils.setException(completePromise, cause);
+ } else {
+ FutureUtils.setValue(completePromise, null);
+ }
+ return BoxedUnit.UNIT;
+ }
+ });
+ }
+ });
+ }
+
+ @VisibleForTesting
+ void closeAndComplete() throws IOException {
+ FutureUtils.result(asyncCloseAndComplete(true));
+ }
+
+ protected Future<Void> asyncCloseAndComplete() {
+ return asyncCloseAndComplete(true);
+ }
+
+ @Override
+ public void close() throws IOException {
+ FutureUtils.result(asyncClose());
+ }
+
+ @Override
+ public Future<Void> asyncClose() {
+ return asyncCloseAndComplete(false);
+ }
+
+ /**
+ * Close the writer and release all the underlying resources
+ */
+ protected Future<Void> closeNoThrow() {
+ Promise<Void> closeFuture;
+ synchronized (this) {
+ if (null != closePromise) {
+ return closePromise;
+ }
+ closeFuture = closePromise = new Promise<Void>();
+ }
+ cancelTruncation();
+ Utils.closeSequence(bkDistributedLogManager.getScheduler(),
+ true, /** ignore close errors **/
+ getCachedLogWriter(),
+ getAllocatedLogWriter(),
+ getCachedWriteHandler()
+ ).proxyTo(closeFuture);
+ return closeFuture;
+ }
+
+ @Override
+ public void abort() throws IOException {
+ FutureUtils.result(asyncAbort());
+ }
+
+ @Override
+ public Future<Void> asyncAbort() {
+ Promise<Void> closeFuture;
+ synchronized (this) {
+ if (null != closePromise) {
+ return closePromise;
+ }
+ closeFuture = closePromise = new Promise<Void>();
+ }
+ cancelTruncation();
+ Abortables.abortSequence(bkDistributedLogManager.getScheduler(),
+ getCachedLogWriter(),
+ getAllocatedLogWriter(),
+ getCachedWriteHandler()).proxyTo(closeFuture);
+ return closeFuture;
+ }
+
+ // used by sync writer
+ protected BKLogSegmentWriter getLedgerWriter(final long startTxId,
+ final boolean allowMaxTxID)
+ throws IOException {
+ Future<BKLogSegmentWriter> logSegmentWriterFuture = asyncGetLedgerWriter(true);
+ BKLogSegmentWriter logSegmentWriter = null;
+ if (null != logSegmentWriterFuture) {
+ logSegmentWriter = FutureUtils.result(logSegmentWriterFuture);
+ }
+ if (null == logSegmentWriter || (shouldStartNewSegment(logSegmentWriter) || forceRolling)) {
+ logSegmentWriter = FutureUtils.result(rollLogSegmentIfNecessary(
+ logSegmentWriter, startTxId, true /* bestEffort */, allowMaxTxID));
+ }
+ return logSegmentWriter;
+ }
+
+ // used by async writer
+ synchronized protected Future<BKLogSegmentWriter> asyncGetLedgerWriter(boolean resetOnError) {
+ final BKLogSegmentWriter ledgerWriter = getCachedLogWriter();
+ Future<BKLogSegmentWriter> ledgerWriterFuture = getCachedLogWriterFuture();
+ if (null == ledgerWriterFuture || null == ledgerWriter) {
+ return null;
+ }
+
+ // Handle the case where the last call to write actually caused an error in the log
+ if ((ledgerWriter.isLogSegmentInError() || forceRecovery) && resetOnError) {
+ // Close the ledger writer so that we will recover and start a new log segment
+ Future<Void> closeFuture;
+ if (ledgerWriter.isLogSegmentInError()) {
+ closeFuture = ledgerWriter.asyncAbort();
+ } else {
+ closeFuture = ledgerWriter.asyncClose();
+ }
+ return closeFuture.flatMap(
+ new AbstractFunction1<Void, Future<BKLogSegmentWriter>>() {
+ @Override
+ public Future<BKLogSegmentWriter> apply(Void result) {
+ removeCachedLogWriter();
+
+ if (ledgerWriter.isLogSegmentInError()) {
+ return Future.value(null);
+ }
+
+ BKLogWriteHandler writeHandler;
+ try {
+ writeHandler = getWriteHandler();
+ } catch (IOException e) {
+ return Future.exception(e);
+ }
+ if (null != writeHandler && forceRecovery) {
+ return writeHandler.completeAndCloseLogSegment(ledgerWriter)
+ .map(new AbstractFunction1<LogSegmentMetadata, BKLogSegmentWriter>() {
+ @Override
+ public BKLogSegmentWriter apply(LogSegmentMetadata completedLogSegment) {
+ return null;
+ }
+ });
+ } else {
+ return Future.value(null);
+ }
+ }
+ });
+ } else {
+ return ledgerWriterFuture;
+ }
+ }
+
+ boolean shouldStartNewSegment(BKLogSegmentWriter ledgerWriter) throws IOException {
+ BKLogWriteHandler writeHandler = getWriteHandler();
+ return null == ledgerWriter || writeHandler.shouldStartNewSegment(ledgerWriter) || forceRolling;
+ }
+
+ private void truncateLogSegmentsIfNecessary(BKLogWriteHandler writeHandler) {
+ boolean truncationEnabled = false;
+
+ long minTimestampToKeep = 0;
+
+ long retentionPeriodInMillis = TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS);
+ if (retentionPeriodInMillis > 0) {
+ minTimestampToKeep = Utils.nowInMillis() - retentionPeriodInMillis;
+ truncationEnabled = true;
+ }
+
+ if (null != minTimestampToKeepOverride) {
+ minTimestampToKeep = minTimestampToKeepOverride;
+ truncationEnabled = true;
+ }
+
+ // skip scheduling if there is task that's already running
+ //
+ synchronized (this) {
+ if (truncationEnabled && ((lastTruncationAttempt == null) || lastTruncationAttempt.isDefined())) {
+ lastTruncationAttempt = writeHandler.purgeLogSegmentsOlderThanTimestamp(minTimestampToKeep);
+ }
+ }
+ }
+
+ private Future<BKLogSegmentWriter> asyncStartNewLogSegment(final BKLogWriteHandler writeHandler,
+ final long startTxId,
+ final boolean allowMaxTxID) {
+ return writeHandler.recoverIncompleteLogSegments()
+ .flatMap(new AbstractFunction1<Long, Future<BKLogSegmentWriter>>() {
+ @Override
+ public Future<BKLogSegmentWriter> apply(Long lastTxId) {
+ return writeHandler.asyncStartLogSegment(startTxId, false, allowMaxTxID)
+ .onSuccess(new AbstractFunction1<BKLogSegmentWriter, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(BKLogSegmentWriter newSegmentWriter) {
+ cacheLogWriter(newSegmentWriter);
+ return BoxedUnit.UNIT;
+ }
+ });
+ }
+ });
+ }
+
+ private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOneWithPermit(
+ final BKLogSegmentWriter oldSegmentWriter,
+ final BKLogWriteHandler writeHandler,
+ final long startTxId,
+ final boolean bestEffort,
+ final boolean allowMaxTxID) {
+ final PermitManager.Permit switchPermit = bkDistributedLogManager.getLogSegmentRollingPermitManager().acquirePermit();
+ if (switchPermit.isAllowed()) {
+ return closeOldLogSegmentAndStartNewOne(
+ oldSegmentWriter,
+ writeHandler,
+ startTxId,
+ bestEffort,
+ allowMaxTxID
+ ).rescue(new Function<Throwable, Future<BKLogSegmentWriter>>() {
+ @Override
+ public Future<BKLogSegmentWriter> apply(Throwable cause) {
+ if (cause instanceof LockingException) {
+ LOG.warn("We lost lock during completeAndClose log segment for {}. Disable ledger rolling until it is recovered : ",
+ writeHandler.getFullyQualifiedName(), cause);
+ bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
+ return Future.value(oldSegmentWriter);
+ } else if (cause instanceof ZKException) {
+ ZKException zke = (ZKException) cause;
+ if (ZKException.isRetryableZKException(zke)) {
+ LOG.warn("Encountered zookeeper connection issues during completeAndClose log segment for {}." +
+ " Disable ledger rolling until it is recovered : {}", writeHandler.getFullyQualifiedName(),
+ zke.getKeeperExceptionCode());
+ bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
+ return Future.value(oldSegmentWriter);
+ }
+ }
+ return Future.exception(cause);
+ }
+ }).ensure(new AbstractFunction0<BoxedUnit>() {
+ @Override
+ public BoxedUnit apply() {
+ bkDistributedLogManager.getLogSegmentRollingPermitManager()
+ .releasePermit(switchPermit);
+ return BoxedUnit.UNIT;
+ }
+ });
+ } else {
+ bkDistributedLogManager.getLogSegmentRollingPermitManager().releasePermit(switchPermit);
+ return Future.value(oldSegmentWriter);
+ }
+ }
+
+ private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOne(
+ final BKLogSegmentWriter oldSegmentWriter,
+ final BKLogWriteHandler writeHandler,
+ final long startTxId,
+ final boolean bestEffort,
+ final boolean allowMaxTxID) {
+ // we switch only when we could allocate a new log segment.
+ BKLogSegmentWriter newSegmentWriter = getAllocatedLogWriter();
+ if (null == newSegmentWriter) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Allocating a new log segment from {} for {}.", startTxId,
+ writeHandler.getFullyQualifiedName());
+ }
+ return writeHandler.asyncStartLogSegment(startTxId, bestEffort, allowMaxTxID)
+ .flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<BKLogSegmentWriter>>() {
+ @Override
+ public Future<BKLogSegmentWriter> apply(BKLogSegmentWriter newSegmentWriter) {
+ if (null == newSegmentWriter) {
+ if (bestEffort) {
+ return Future.value(oldSegmentWriter);
+ } else {
+ return Future.exception(
+ new UnexpectedException("StartLogSegment returns null for bestEffort rolling"));
+ }
+ }
+ cacheAllocatedLogWriter(newSegmentWriter);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Allocated a new log segment from {} for {}.", startTxId,
+ writeHandler.getFullyQualifiedName());
+ }
+ return completeOldSegmentAndCacheNewLogSegmentWriter(oldSegmentWriter, newSegmentWriter);
+ }
+ });
+ } else {
+ return completeOldSegmentAndCacheNewLogSegmentWriter(oldSegmentWriter, newSegmentWriter);
+ }
+ }
+
+ private Future<BKLogSegmentWriter> completeOldSegmentAndCacheNewLogSegmentWriter(
+ BKLogSegmentWriter oldSegmentWriter,
+ final BKLogSegmentWriter newSegmentWriter) {
+ final Promise<BKLogSegmentWriter> completePromise = new Promise<BKLogSegmentWriter>();
+ // complete the old log segment
+ writeHandler.completeAndCloseLogSegment(oldSegmentWriter)
+ .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+
+ @Override
+ public void onSuccess(LogSegmentMetadata value) {
+ cacheLogWriter(newSegmentWriter);
+ removeAllocatedLogWriter();
+ FutureUtils.setValue(completePromise, newSegmentWriter);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ FutureUtils.setException(completePromise, cause);
+ }
+ });
+ return completePromise;
+ }
+
+ synchronized protected Future<BKLogSegmentWriter> rollLogSegmentIfNecessary(
+ final BKLogSegmentWriter segmentWriter,
+ long startTxId,
+ boolean bestEffort,
+ boolean allowMaxTxID) {
+ final BKLogWriteHandler writeHandler;
+ try {
+ writeHandler = getWriteHandler();
+ } catch (IOException e) {
+ return Future.exception(e);
+ }
+ Future<BKLogSegmentWriter> rollPromise;
+ if (null != segmentWriter && (writeHandler.shouldStartNewSegment(segmentWriter) || forceRolling)) {
+ rollPromise = closeOldLogSegmentAndStartNewOneWithPermit(
+ segmentWriter, writeHandler, startTxId, bestEffort, allowMaxTxID);
+ } else if (null == segmentWriter) {
+ rollPromise = asyncStartNewLogSegment(writeHandler, startTxId, allowMaxTxID);
+ } else {
+ rollPromise = Future.value(segmentWriter);
+ }
+ return rollPromise.map(new AbstractFunction1<BKLogSegmentWriter, BKLogSegmentWriter>() {
+ @Override
+ public BKLogSegmentWriter apply(BKLogSegmentWriter newSegmentWriter) {
+ if (segmentWriter == newSegmentWriter) {
+ return newSegmentWriter;
+ }
+ truncateLogSegmentsIfNecessary(writeHandler);
+ return newSegmentWriter;
+ }
+ });
+ }
+
+ protected synchronized void checkClosedOrInError(String operation) throws AlreadyClosedException {
+ if (null != closePromise) {
+ LOG.error("Executing " + operation + " on already closed Log Writer");
+ throw new AlreadyClosedException("Executing " + operation + " on already closed Log Writer");
+ }
+ }
+
+ @VisibleForTesting
+ public void setForceRolling(boolean forceRolling) {
+ this.forceRolling = forceRolling;
+ }
+
+ @VisibleForTesting
+ public synchronized void overRideMinTimeStampToKeep(Long minTimestampToKeepOverride) {
+ this.minTimestampToKeepOverride = minTimestampToKeepOverride;
+ }
+
+ protected synchronized void cancelTruncation() {
+ if (null != lastTruncationAttempt) {
+ FutureUtils.cancel(lastTruncationAttempt);
+ lastTruncationAttempt = null;
+ }
+ }
+
+ @VisibleForTesting
+ public synchronized void setForceRecovery(boolean forceRecovery) {
+ this.forceRecovery = forceRecovery;
+ }
+
+}
[36/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
deleted file mode 100644
index a3959b0..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
+++ /dev/null
@@ -1,1106 +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.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.callback.LogSegmentListener;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.LogEmptyException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.function.CloseAsyncCloseableFunction;
-import com.twitter.distributedlog.function.GetVersionedValueFunction;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-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.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.LogStreamMetadataStore;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.stats.BroadCastStatsLogger;
-import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.distributedlog.util.Allocator;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.MonitoredFuturePool;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.PermitManager;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.ExceptionalFunction;
-import com.twitter.util.ExceptionalFunction0;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-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.StatsLogger;
-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;
-import java.util.List;
-import java.util.Set;
-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>
- * <li> `log_writer/*`: all asynchronous writer related metrics are exposed under scope `log_writer`.
- * See {@link BKAsyncLogWriter} for detail stats.
- * <li> `async_reader/*`: all asyncrhonous reader related metrics are exposed under scope `async_reader`.
- * See {@link BKAsyncLogReader} for detail stats.
- * <li> `writer_future_pool/*`: metrics about the future pools that used by writers are exposed under
- * scope `writer_future_pool`. See {@link MonitoredFuturePool} for detail stats.
- * <li> `reader_future_pool/*`: metrics about the future pools that used by readers are exposed under
- * scope `reader_future_pool`. See {@link MonitoredFuturePool} for detail stats.
- * <li> `lock/*`: metrics about the locks used by writers. See {@link ZKDistributedLock} for detail
- * stats.
- * <li> `read_lock/*`: metrics about the locks used by readers. See {@link ZKDistributedLock} for
- * detail stats.
- * <li> `logsegments/*`: metrics about basic operations on log segments. See {@link BKLogHandler} for details.
- * <li> `segments/*`: metrics about write operations on log segments. See {@link BKLogWriteHandler} for details.
- * <li> `readahead_worker/*`: metrics about readahead workers used by readers. See {@link BKLogReadHandler}
- * for details.
- * </ul>
- */
-class BKDistributedLogManager implements DistributedLogManager {
- static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogManager.class);
-
- static final Function<LogRecordWithDLSN, Long> RECORD_2_TXID_FUNCTION =
- new Function<LogRecordWithDLSN, Long>() {
- @Override
- public Long apply(LogRecordWithDLSN record) {
- return record.getTransactionId();
- }
- };
-
- static final Function<LogRecordWithDLSN, DLSN> RECORD_2_DLSN_FUNCTION =
- new Function<LogRecordWithDLSN, DLSN>() {
- @Override
- public DLSN apply(LogRecordWithDLSN record) {
- return record.getDlsn();
- }
- };
-
- 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 FeatureProvider featureProvider;
- private final AsyncFailureInjector failureInjector;
- private final StatsLogger statsLogger;
- private final StatsLogger perLogStatsLogger;
- final AlertStatsLogger alertStatsLogger;
-
- // log segment metadata cache
- private final LogSegmentMetadataCache logSegmentMetadataCache;
-
- //
- // Writer Related Variables
- //
- private final PermitLimiter writeLimiter;
-
- //
- // Reader Related Variables
- ///
- // read handler for listener.
- private BKLogReadHandler readHandlerForListener = null;
- private final PendingReaders pendingReaders;
-
- // 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 driver namespace driver
- * @param logSegmentMetadataCache log segment metadata cache
- * @param scheduler ordered scheduled used by readers and writers
- * @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 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
- */
- BKDistributedLogManager(String name,
- DistributedLogConfiguration conf,
- DynamicDistributedLogConfiguration dynConf,
- URI uri,
- NamespaceDriver driver,
- LogSegmentMetadataCache logSegmentMetadataCache,
- OrderedScheduler scheduler,
- String clientId,
- Integer regionId,
- PermitLimiter writeLimiter,
- FeatureProvider featureProvider,
- AsyncFailureInjector failureInjector,
- StatsLogger 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.statsLogger = statsLogger;
- this.perLogStatsLogger = BroadCastStatsLogger.masterslave(perLogStatsLogger, statsLogger);
- this.pendingReaders = new PendingReaders(scheduler);
- this.regionId = regionId;
- this.clientId = clientId;
- this.streamIdentifier = conf.getUnpartitionedStreamName();
- this.writeLimiter = writeLimiter;
- // Feature Provider
- this.featureProvider = featureProvider;
- // Failure Injector
- this.failureInjector = failureInjector;
- // Stats
- this.alertStatsLogger = new AlertStatsLogger(this.perLogStatsLogger, "dl_alert");
- this.resourcesCloseable = resourcesCloseable;
- }
-
- @Override
- public String getStreamName() {
- return name;
- }
-
- @Override
- public NamespaceDriver getNamespaceDriver() {
- return driver;
- }
-
- URI getUri() {
- return uri;
- }
-
- DistributedLogConfiguration getConf() {
- return conf;
- }
-
- OrderedScheduler getScheduler() {
- return scheduler;
- }
-
- AsyncFailureInjector getFailureInjector() {
- return failureInjector;
- }
-
- //
- // Test Methods
- //
-
- @VisibleForTesting
- LogStreamMetadataStore getWriterMetadataStore() {
- return driver.getLogStreamMetadataStore(WRITER);
- }
-
- @VisibleForTesting
- LogSegmentEntryStore getReaderEntryStore() {
- return driver.getLogSegmentEntryStore(READER);
- }
-
- @VisibleForTesting
- FeatureProvider getFeatureProvider() {
- return this.featureProvider;
- }
-
- private synchronized BKLogReadHandler getReadHandlerAndRegisterListener(
- boolean create, LogSegmentListener listener) {
- if (null == readHandlerForListener && create) {
- readHandlerForListener = createReadHandler();
- readHandlerForListener.registerListener(listener);
- // start fetch the log segments after created the listener
- readHandlerForListener.asyncStartFetchLogSegments();
- return readHandlerForListener;
- }
- if (null != readHandlerForListener && null != listener) {
- readHandlerForListener.registerListener(listener);
- }
- return readHandlerForListener;
- }
-
- @Override
- public List<LogSegmentMetadata> getLogSegments() throws IOException {
- return FutureUtils.result(getLogSegmentsAsync());
- }
-
- protected Future<List<LogSegmentMetadata>> getLogSegmentsAsync() {
- final BKLogReadHandler readHandler = createReadHandler();
- return readHandler.readLogSegmentsFromStore(
- LogSegmentMetadata.COMPARATOR,
- LogSegmentFilter.DEFAULT_FILTER,
- null)
- .map(GetVersionedValueFunction.GET_LOGSEGMENT_LIST_FUNC)
- .ensure(CloseAsyncCloseableFunction.of(readHandler));
- }
-
- @Override
- public void registerListener(LogSegmentListener listener) throws IOException {
- getReadHandlerAndRegisterListener(true, listener);
- }
-
- @Override
- public synchronized void unregisterListener(LogSegmentListener listener) {
- if (null != readHandlerForListener) {
- readHandlerForListener.unregisterListener(listener);
- }
- }
-
- public void checkClosedOrInError(String operation) throws AlreadyClosedException {
- synchronized (this) {
- if (null != closePromise) {
- throw new AlreadyClosedException("Executing " + operation + " on already closed DistributedLogManager");
- }
- }
- }
-
- // Create Read Handler
-
- synchronized BKLogReadHandler createReadHandler() {
- Optional<String> subscriberId = Optional.absent();
- return createReadHandler(subscriberId, false);
- }
-
- synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId) {
- return createReadHandler(subscriberId, false);
- }
-
- synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId,
- boolean isHandleForReading) {
- return createReadHandler(
- subscriberId,
- null,
- isHandleForReading);
- }
-
- synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId,
- AsyncNotification notification,
- boolean isHandleForReading) {
- LogMetadataForReader logMetadata = LogMetadataForReader.of(uri, name, streamIdentifier);
- return new BKLogReadHandler(
- logMetadata,
- subscriberId,
- conf,
- dynConf,
- driver.getLogStreamMetadataStore(READER),
- logSegmentMetadataCache,
- driver.getLogSegmentEntryStore(READER),
- scheduler,
- alertStatsLogger,
- statsLogger,
- perLogStatsLogger,
- clientId,
- notification,
- isHandleForReading);
- }
-
- // Create Ledger Allocator
-
-
-
- // Create Write Handler
-
- public BKLogWriteHandler createWriteHandler(boolean lockHandler)
- throws IOException {
- return FutureUtils.result(asyncCreateWriteHandler(lockHandler));
- }
-
- Future<BKLogWriteHandler> asyncCreateWriteHandler(final boolean lockHandler) {
- // Fetching Log Metadata (create if not exists)
- return driver.getLogStreamMetadataStore(WRITER).getLog(
- uri,
- name,
- true,
- conf.getCreateStreamIfNotExists()
- ).flatMap(new AbstractFunction1<LogMetadataForWriter, Future<BKLogWriteHandler>>() {
- @Override
- public Future<BKLogWriteHandler> apply(LogMetadataForWriter logMetadata) {
- Promise<BKLogWriteHandler> createPromise = new Promise<BKLogWriteHandler>();
- createWriteHandler(logMetadata, lockHandler, createPromise);
- return createPromise;
- }
- });
- }
-
- private void createWriteHandler(LogMetadataForWriter logMetadata,
- boolean lockHandler,
- final Promise<BKLogWriteHandler> createPromise) {
- // Build the locks
- DistributedLock lock;
- if (conf.isWriteLockEnabled()) {
- lock = driver.getLogStreamMetadataStore(WRITER).createWriteLock(logMetadata);
- } else {
- lock = NopDistributedLock.INSTANCE;
- }
-
- Allocator<LogSegmentEntryWriter, Object> segmentAllocator;
- try {
- segmentAllocator = driver.getLogSegmentEntryStore(WRITER)
- .newLogSegmentAllocator(logMetadata, dynConf);
- } catch (IOException ioe) {
- FutureUtils.setException(createPromise, ioe);
- return;
- }
-
- // Make sure writer handler created before resources are initialized
- final BKLogWriteHandler writeHandler = new BKLogWriteHandler(
- logMetadata,
- conf,
- driver.getLogStreamMetadataStore(WRITER),
- logSegmentMetadataCache,
- driver.getLogSegmentEntryStore(WRITER),
- scheduler,
- segmentAllocator,
- statsLogger,
- perLogStatsLogger,
- alertStatsLogger,
- clientId,
- regionId,
- writeLimiter,
- featureProvider,
- dynConf,
- lock);
- if (lockHandler) {
- writeHandler.lockHandler().addEventListener(new FutureEventListener<DistributedLock>() {
- @Override
- public void onSuccess(DistributedLock lock) {
- FutureUtils.setValue(createPromise, writeHandler);
- }
-
- @Override
- public void onFailure(final Throwable cause) {
- writeHandler.asyncClose().ensure(new AbstractFunction0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- FutureUtils.setException(createPromise, cause);
- return BoxedUnit.UNIT;
- }
- });
- }
- });
- } else {
- FutureUtils.setValue(createPromise, writeHandler);
- }
- }
-
- PermitManager getLogSegmentRollingPermitManager() {
- return driver.getLogStreamMetadataStore(WRITER).getPermitManager();
- }
-
- <T> Future<T> processReaderOperation(final Function<BKLogReadHandler, Future<T>> func) {
- return scheduler.apply(new ExceptionalFunction0<BKLogReadHandler>() {
- @Override
- public BKLogReadHandler applyE() throws Throwable {
- return getReadHandlerAndRegisterListener(true, null);
- }
- }).flatMap(new ExceptionalFunction<BKLogReadHandler, Future<T>>() {
- @Override
- public Future<T> applyE(final BKLogReadHandler readHandler) throws Throwable {
- return func.apply(readHandler);
- }
- });
- }
-
- /**
- * Check if an end of stream marker was added to the stream
- * A stream with an end of stream marker cannot be appended to
- *
- * @return true if the marker was added to the stream, false otherwise
- */
- @Override
- public boolean isEndOfStreamMarked() throws IOException {
- checkClosedOrInError("isEndOfStreamMarked");
- long lastTxId = FutureUtils.result(getLastLogRecordAsyncInternal(false, true)).getTransactionId();
- return lastTxId == DistributedLogConstants.MAX_TXID;
- }
-
- /**
- * Begin appending to the end of the log stream which is being treated as a sequence of bytes
- *
- * @return the writer interface to generate log records
- */
- public AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException {
- long position;
- try {
- position = FutureUtils.result(getLastLogRecordAsyncInternal(true, false)).getTransactionId();
- if (DistributedLogConstants.INVALID_TXID == position ||
- DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID == position) {
- position = 0;
- }
- } catch (LogEmptyException ex) {
- position = 0;
- } catch (LogNotFoundException ex) {
- position = 0;
- }
- return new AppendOnlyStreamWriter(startAsyncLogSegmentNonPartitioned(), position);
- }
-
- /**
- * Get a reader to read a log stream as a sequence of bytes
- *
- * @return the writer interface to generate log records
- */
- public AppendOnlyStreamReader getAppendOnlyStreamReader() throws IOException {
- return new AppendOnlyStreamReader(this);
- }
-
- /**
- * Begin writing to the log stream identified by the name
- *
- * @return the writer interface to generate log records
- */
- @Override
- public BKSyncLogWriter startLogSegmentNonPartitioned() throws IOException {
- checkClosedOrInError("startLogSegmentNonPartitioned");
- BKSyncLogWriter writer = new BKSyncLogWriter(conf, dynConf, this);
- boolean success = false;
- try {
- writer.createAndCacheWriteHandler();
- BKLogWriteHandler writeHandler = writer.getWriteHandler();
- FutureUtils.result(writeHandler.lockHandler());
- success = true;
- return writer;
- } finally {
- if (!success) {
- writer.abort();
- }
- }
- }
-
- /**
- * Begin writing to the log stream identified by the name
- *
- * @return the writer interface to generate log records
- */
- @Override
- public BKAsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException {
- return (BKAsyncLogWriter) FutureUtils.result(openAsyncLogWriter());
- }
-
- @Override
- public Future<AsyncLogWriter> openAsyncLogWriter() {
- try {
- checkClosedOrInError("startLogSegmentNonPartitioned");
- } catch (AlreadyClosedException e) {
- return Future.exception(e);
- }
-
- Future<BKLogWriteHandler> createWriteHandleFuture;
- synchronized (this) {
- // 1. create the locked write handler
- createWriteHandleFuture = asyncCreateWriteHandler(true);
- }
- return createWriteHandleFuture.flatMap(new AbstractFunction1<BKLogWriteHandler, Future<AsyncLogWriter>>() {
- @Override
- public Future<AsyncLogWriter> apply(final BKLogWriteHandler writeHandler) {
- final BKAsyncLogWriter writer;
- synchronized (BKDistributedLogManager.this) {
- // 2. create the writer with the handler
- writer = new BKAsyncLogWriter(
- conf,
- dynConf,
- BKDistributedLogManager.this,
- writeHandler,
- featureProvider,
- statsLogger);
- }
- // 3. recover the incomplete log segments
- return writeHandler.recoverIncompleteLogSegments()
- .map(new AbstractFunction1<Long, AsyncLogWriter>() {
- @Override
- public AsyncLogWriter apply(Long lastTxId) {
- // 4. update last tx id if successfully recovered
- writer.setLastTxId(lastTxId);
- return writer;
- }
- }).onFailure(new AbstractFunction1<Throwable, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Throwable cause) {
- // 5. close the writer if recovery failed
- writer.asyncAbort();
- return BoxedUnit.UNIT;
- }
- });
- }
- });
- }
-
- @Override
- public Future<DLSN> getDLSNNotLessThanTxId(final long fromTxnId) {
- return getLogSegmentsAsync().flatMap(new AbstractFunction1<List<LogSegmentMetadata>, Future<DLSN>>() {
- @Override
- public Future<DLSN> apply(List<LogSegmentMetadata> segments) {
- return getDLSNNotLessThanTxId(fromTxnId, segments);
- }
- });
- }
-
- private Future<DLSN> getDLSNNotLessThanTxId(long fromTxnId,
- final List<LogSegmentMetadata> segments) {
- if (segments.isEmpty()) {
- return getLastDLSNAsync();
- }
- final int segmentIdx = DLUtils.findLogSegmentNotLessThanTxnId(segments, fromTxnId);
- if (segmentIdx < 0) {
- return Future.value(new DLSN(segments.get(0).getLogSegmentSequenceNumber(), 0L, 0L));
- }
- return getDLSNNotLessThanTxIdInSegment(
- fromTxnId,
- segmentIdx,
- segments,
- driver.getLogSegmentEntryStore(READER)
- );
- }
-
- private Future<DLSN> getDLSNNotLessThanTxIdInSegment(final long fromTxnId,
- final int segmentIdx,
- final List<LogSegmentMetadata> segments,
- final LogSegmentEntryStore entryStore) {
- final LogSegmentMetadata segment = segments.get(segmentIdx);
- return ReadUtils.getLogRecordNotLessThanTxId(
- name,
- segment,
- fromTxnId,
- scheduler,
- entryStore,
- Math.max(2, dynConf.getReadAheadBatchSize())
- ).flatMap(new AbstractFunction1<Optional<LogRecordWithDLSN>, Future<DLSN>>() {
- @Override
- public Future<DLSN> apply(Optional<LogRecordWithDLSN> foundRecord) {
- if (foundRecord.isPresent()) {
- return Future.value(foundRecord.get().getDlsn());
- }
- if ((segments.size() - 1) == segmentIdx) {
- return getLastLogRecordAsync().map(new AbstractFunction1<LogRecordWithDLSN, DLSN>() {
- @Override
- public DLSN apply(LogRecordWithDLSN record) {
- if (record.getTransactionId() >= fromTxnId) {
- return record.getDlsn();
- }
- return record.getDlsn().getNextDLSN();
- }
- });
- } else {
- return getDLSNNotLessThanTxIdInSegment(
- fromTxnId,
- segmentIdx + 1,
- segments,
- entryStore);
- }
- }
- });
- }
-
- /**
- * Get the input stream starting with fromTxnId for the specified log
- *
- * @param fromTxnId - the first transaction id we want to read
- * @return the stream starting with transaction fromTxnId
- * @throws IOException if a stream cannot be found.
- */
- @Override
- public LogReader getInputStream(long fromTxnId)
- throws IOException {
- return getInputStreamInternal(fromTxnId);
- }
-
- @Override
- public LogReader getInputStream(DLSN fromDLSN) throws IOException {
- return getInputStreamInternal(fromDLSN, Optional.<Long>absent());
- }
-
- @Override
- public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException {
- return FutureUtils.result(openAsyncLogReader(fromTxnId));
- }
-
- /**
- * Opening a log reader positioning by transaction id <code>fromTxnId</code>.
- *
- * <p>
- * - retrieve log segments for the stream
- * - if the log segment list is empty, positioning by the last dlsn
- * - otherwise, find the first log segment that contains the records whose transaction ids are not less than
- * the provided transaction id <code>fromTxnId</code>
- * - if all log segments' records' transaction ids are more than <code>fromTxnId</code>, positioning
- * on the first record.
- * - otherwise, search the log segment to find the log record
- * - if the log record is found, positioning the reader by that found record's dlsn
- * - otherwise, positioning by the last dlsn
- * </p>
- *
- * @see DLUtils#findLogSegmentNotLessThanTxnId(List, long)
- * @see ReadUtils#getLogRecordNotLessThanTxId(String, LogSegmentMetadata, long, ExecutorService, LogSegmentEntryStore, int)
- * @param fromTxnId
- * transaction id to start reading from
- * @return future representing the open result.
- */
- @Override
- public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId) {
- final Promise<DLSN> dlsnPromise = new Promise<DLSN>();
- getDLSNNotLessThanTxId(fromTxnId).addEventListener(new FutureEventListener<DLSN>() {
-
- @Override
- public void onSuccess(DLSN dlsn) {
- dlsnPromise.setValue(dlsn);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- if (cause instanceof LogEmptyException) {
- dlsnPromise.setValue(DLSN.InitialDLSN);
- } else {
- dlsnPromise.setException(cause);
- }
- }
- });
- return dlsnPromise.flatMap(new AbstractFunction1<DLSN, Future<AsyncLogReader>>() {
- @Override
- public Future<AsyncLogReader> apply(DLSN dlsn) {
- return openAsyncLogReader(dlsn);
- }
- });
- }
-
- @Override
- public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException {
- return FutureUtils.result(openAsyncLogReader(fromDLSN));
- }
-
- @Override
- public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN) {
- Optional<String> subscriberId = Optional.absent();
- AsyncLogReader reader = new BKAsyncLogReader(
- this,
- scheduler,
- fromDLSN,
- subscriberId,
- false,
- statsLogger);
- pendingReaders.add(reader);
- return Future.value(reader);
- }
-
- /**
- * Note the lock here is a sort of elective exclusive lock. I.e. acquiring this lock will only prevent other
- * people who try to acquire the lock from reading from the stream. Normal readers (and writers) will not be
- * blocked.
- */
- @Override
- public Future<AsyncLogReader> getAsyncLogReaderWithLock(final DLSN fromDLSN) {
- Optional<String> subscriberId = Optional.absent();
- return getAsyncLogReaderWithLock(Optional.of(fromDLSN), subscriberId);
- }
-
- @Override
- public Future<AsyncLogReader> getAsyncLogReaderWithLock(final DLSN fromDLSN, final String subscriberId) {
- return getAsyncLogReaderWithLock(Optional.of(fromDLSN), Optional.of(subscriberId));
- }
-
- @Override
- public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId) {
- Optional<DLSN> fromDLSN = Optional.absent();
- return getAsyncLogReaderWithLock(fromDLSN, Optional.of(subscriberId));
- }
-
- protected Future<AsyncLogReader> getAsyncLogReaderWithLock(final Optional<DLSN> fromDLSN,
- final Optional<String> subscriberId) {
- if (!fromDLSN.isPresent() && !subscriberId.isPresent()) {
- return Future.exception(new UnexpectedException("Neither from dlsn nor subscriber id is provided."));
- }
- final BKAsyncLogReader reader = new BKAsyncLogReader(
- BKDistributedLogManager.this,
- scheduler,
- fromDLSN.isPresent() ? fromDLSN.get() : DLSN.InitialDLSN,
- subscriberId,
- false,
- statsLogger);
- pendingReaders.add(reader);
- final Future<Void> lockFuture = reader.lockStream();
- final Promise<AsyncLogReader> createPromise = new Promise<AsyncLogReader>(
- new Function<Throwable, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Throwable cause) {
- // cancel the lock when the creation future is cancelled
- lockFuture.cancel();
- return BoxedUnit.UNIT;
- }
- });
- // lock the stream - fetch the last commit position on success
- lockFuture.flatMap(new Function<Void, Future<AsyncLogReader>>() {
- @Override
- public Future<AsyncLogReader> apply(Void complete) {
- if (fromDLSN.isPresent()) {
- return Future.value((AsyncLogReader) reader);
- }
- LOG.info("Reader {} @ {} reading last commit position from subscription store after acquired lock.",
- subscriberId.get(), name);
- // we acquired lock
- 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 {}.",
- new Object[] { subscriberId.get(), name, lastCommitPosition });
- reader.setStartDLSN(lastCommitPosition);
- return reader;
- }
- });
- }
- }).addEventListener(new FutureEventListener<AsyncLogReader>() {
- @Override
- public void onSuccess(AsyncLogReader r) {
- pendingReaders.remove(reader);
- FutureUtils.setValue(createPromise, r);
- }
-
- @Override
- public void onFailure(final Throwable cause) {
- pendingReaders.remove(reader);
- reader.asyncClose().ensure(new AbstractFunction0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- FutureUtils.setException(createPromise, cause);
- return BoxedUnit.UNIT;
- }
- });
- }
- });
- return createPromise;
- }
-
- /**
- * Get the input stream starting with fromTxnId for the specified log
- *
- * @param fromTxnId
- * transaction id to start reading from
- * @return log reader
- * @throws IOException
- */
- LogReader getInputStreamInternal(long fromTxnId)
- throws IOException {
- DLSN fromDLSN;
- try {
- fromDLSN = FutureUtils.result(getDLSNNotLessThanTxId(fromTxnId));
- } catch (LogEmptyException lee) {
- fromDLSN = DLSN.InitialDLSN;
- }
- return getInputStreamInternal(fromDLSN, Optional.of(fromTxnId));
- }
-
- LogReader getInputStreamInternal(DLSN fromDLSN, Optional<Long> fromTxnId)
- throws IOException {
- LOG.info("Create sync reader starting from {}", fromDLSN);
- checkClosedOrInError("getInputStream");
- return new BKSyncLogReader(
- conf,
- this,
- fromDLSN,
- fromTxnId,
- statsLogger);
- }
-
- /**
- * Get the last log record in the stream
- *
- * @return the last log record in the stream
- * @throws java.io.IOException if a stream cannot be found.
- */
- @Override
- public LogRecordWithDLSN getLastLogRecord() throws IOException {
- checkClosedOrInError("getLastLogRecord");
- return FutureUtils.result(getLastLogRecordAsync());
- }
-
- @Override
- public long getFirstTxId() throws IOException {
- checkClosedOrInError("getFirstTxId");
- return FutureUtils.result(getFirstRecordAsyncInternal()).getTransactionId();
- }
-
- @Override
- public long getLastTxId() throws IOException {
- checkClosedOrInError("getLastTxId");
- return FutureUtils.result(getLastTxIdAsync());
- }
-
- @Override
- public DLSN getLastDLSN() throws IOException {
- checkClosedOrInError("getLastDLSN");
- return FutureUtils.result(getLastLogRecordAsyncInternal(false, false)).getDlsn();
- }
-
- /**
- * Get Latest log record in the log
- *
- * @return latest log record
- */
- @Override
- public Future<LogRecordWithDLSN> getLastLogRecordAsync() {
- return getLastLogRecordAsyncInternal(false, false);
- }
-
- private Future<LogRecordWithDLSN> getLastLogRecordAsyncInternal(final boolean recover,
- final boolean includeEndOfStream) {
- return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
- @Override
- public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
- return ledgerHandler.getLastLogRecordAsync(recover, includeEndOfStream);
- }
- });
- }
-
- /**
- * Get Latest Transaction Id in the log
- *
- * @return latest transaction id
- */
- @Override
- public Future<Long> getLastTxIdAsync() {
- return getLastLogRecordAsyncInternal(false, false)
- .map(RECORD_2_TXID_FUNCTION);
- }
-
- /**
- * Get first DLSN in the log.
- *
- * @return first dlsn in the stream
- */
- @Override
- public Future<DLSN> getFirstDLSNAsync() {
- return getFirstRecordAsyncInternal().map(RECORD_2_DLSN_FUNCTION);
- }
-
- private Future<LogRecordWithDLSN> getFirstRecordAsyncInternal() {
- return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
- @Override
- public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
- return ledgerHandler.asyncGetFirstLogRecord();
- }
- });
- }
-
- /**
- * Get Latest DLSN in the log.
- *
- * @return latest transaction id
- */
- @Override
- public Future<DLSN> getLastDLSNAsync() {
- return getLastLogRecordAsyncInternal(false, false)
- .map(RECORD_2_DLSN_FUNCTION);
- }
-
- /**
- * Get the number of log records in the active portion of the log
- * Any log segments that have already been truncated will not be included
- *
- * @return number of log records
- * @throws IOException
- */
- @Override
- public long getLogRecordCount() throws IOException {
- checkClosedOrInError("getLogRecordCount");
- return FutureUtils.result(getLogRecordCountAsync(DLSN.InitialDLSN));
- }
-
- /**
- * Get the number of log records in the active portion of the log asynchronously.
- * Any log segments that have already been truncated will not be included
- *
- * @return future number of log records
- * @throws IOException
- */
- @Override
- public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN) {
- return processReaderOperation(new Function<BKLogReadHandler, Future<Long>>() {
- @Override
- public Future<Long> apply(BKLogReadHandler ledgerHandler) {
- return ledgerHandler.asyncGetLogRecordCount(beginDLSN);
- }
- });
- }
-
- @Override
- public void recover() throws IOException {
- recoverInternal(conf.getUnpartitionedStreamName());
- }
-
- /**
- * Recover a specified stream within the log container
- * The writer implicitly recovers a topic when it resumes writing.
- * This allows applications to recover a container explicitly so
- * that application may read a fully recovered log before resuming
- * the writes
- *
- * @throws IOException if the recovery fails
- */
- private void recoverInternal(String streamIdentifier) throws IOException {
- checkClosedOrInError("recoverInternal");
- BKLogWriteHandler ledgerHandler = createWriteHandler(true);
- try {
- FutureUtils.result(ledgerHandler.recoverIncompleteLogSegments());
- } finally {
- Utils.closeQuietly(ledgerHandler);
- }
- }
-
- /**
- * Delete all the partitions of the specified log
- *
- * @throws IOException if the deletion fails
- */
- @Override
- public void delete() throws IOException {
- FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
- .deleteLog(uri, getStreamName()));
- }
-
- /**
- * The DistributedLogManager may archive/purge any logs for transactionId
- * less than or equal to minImageTxId.
- * This is to be used only when the client explicitly manages deletion. If
- * the cleanup policy is based on sliding time window, then this method need
- * not be called.
- *
- * @param minTxIdToKeep the earliest txid that must be retained
- * @throws IOException if purging fails
- */
- @Override
- public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException {
- Preconditions.checkArgument(minTxIdToKeep > 0, "Invalid transaction id " + minTxIdToKeep);
- checkClosedOrInError("purgeLogSegmentsOlderThan");
- BKLogWriteHandler ledgerHandler = createWriteHandler(true);
- try {
- LOG.info("Purging logs for {} older than {}", ledgerHandler.getFullyQualifiedName(), minTxIdToKeep);
- FutureUtils.result(ledgerHandler.purgeLogSegmentsOlderThanTxnId(minTxIdToKeep));
- } finally {
- Utils.closeQuietly(ledgerHandler);
- }
- }
-
- static class PendingReaders implements AsyncCloseable {
-
- final ExecutorService executorService;
- final Set<AsyncCloseable> readers = new HashSet<AsyncCloseable>();
-
- PendingReaders(ExecutorService executorService) {
- this.executorService = executorService;
- }
-
- public synchronized void remove(AsyncCloseable reader) {
- readers.remove(reader);
- }
-
- public synchronized void add(AsyncCloseable reader) {
- readers.add(reader);
- }
-
- @Override
- public Future<Void> asyncClose() {
- return Utils.closeSequence(executorService, true, readers.toArray(new AsyncLogReader[readers.size()]))
- .onSuccess(new AbstractFunction1<Void, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Void value) {
- readers.clear();
- return BoxedUnit.UNIT;
- }
- });
- }
- };
-
- /**
- * Close the distributed log manager, freeing any resources it may hold.
- */
- @Override
- public Future<Void> asyncClose() {
- Promise<Void> closeFuture;
- BKLogReadHandler readHandlerToClose;
- synchronized (this) {
- if (null != closePromise) {
- return closePromise;
- }
- closeFuture = closePromise = new Promise<Void>();
- readHandlerToClose = readHandlerForListener;
- }
-
- Future<Void> closeResult = Utils.closeSequence(null, true,
- readHandlerToClose,
- pendingReaders,
- resourcesCloseable.or(AsyncCloseable.NULL));
- closeResult.proxyTo(closeFuture);
- return closeFuture;
- }
-
- @Override
- public void close() throws IOException {
- FutureUtils.result(asyncClose());
- }
-
- @Override
- public String toString() {
- return String.format("DLM:%s:%s", getUri(), getStreamName());
- }
-
- public void raiseAlert(String msg, Object... args) {
- alertStatsLogger.raise(msg, args);
- }
-
- @Override
- public SubscriptionsStore getSubscriptionsStore() {
- return driver.getSubscriptionsStore(getStreamName());
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
deleted file mode 100644
index a8b1f77..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
+++ /dev/null
@@ -1,320 +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.google.common.base.Ticker;
-import com.twitter.distributedlog.acl.AccessControlManager;
-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.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.util.ConfUtils;
-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.Utils;
-import org.apache.bookkeeper.feature.FeatureProvider;
-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.Iterator;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-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
- * zookeeper for metadata storage and bookkeeper for data storage.
- * <h3>Metrics</h3>
- *
- * <h4>ZooKeeper Client</h4>
- * See {@link ZooKeeperClient} for detail sub-stats.
- * <ul>
- * <li> `scope`/dlzk_factory_writer_shared/* : stats about the zookeeper client shared by all DL writers.
- * <li> `scope`/dlzk_factory_reader_shared/* : stats about the zookeeper client shared by all DL readers.
- * <li> `scope`/bkzk_factory_writer_shared/* : stats about the zookeeper client used by bookkeeper client
- * shared by all DL writers.
- * <li> `scope`/bkzk_factory_reader_shared/* : stats about the zookeeper client used by bookkeeper client
- * shared by all DL readers.
- * </ul>
- *
- * <h4>BookKeeper Client</h4>
- * BookKeeper client stats are exposed directly to current scope. See {@link BookKeeperClient} for detail stats.
- *
- * <h4>Utils</h4>
- * <ul>
- * <li> `scope`/factory/thread_pool/* : stats about the ordered scheduler used by this namespace.
- * See {@link OrderedScheduler}.
- * <li> `scope`/factory/readahead_thread_pool/* : stats about the readahead thread pool executor
- * used by this namespace. See {@link MonitoredScheduledThreadPoolExecutor}.
- * <li> `scope`/writeLimiter/* : stats about the global write limiter used by this namespace.
- * See {@link PermitLimiter}.
- * </ul>
- *
- * <h4>DistributedLogManager</h4>
- *
- * All the core stats about reader and writer are exposed under current scope via {@link BKDistributedLogManager}.
- */
-public class BKDistributedLogNamespace implements DistributedLogNamespace {
- static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogNamespace.class);
-
- private final String clientId;
- private final int regionId;
- private final DistributedLogConfiguration conf;
- private final URI namespace;
- // namespace driver
- private final NamespaceDriver driver;
- // resources
- private final OrderedScheduler scheduler;
- private final PermitLimiter writeLimiter;
- private final AsyncFailureInjector failureInjector;
- // log segment metadata store
- private final LogSegmentMetadataCache logSegmentMetadataCache;
- // feature provider
- private final FeatureProvider featureProvider;
- // Stats Loggers
- private final StatsLogger statsLogger;
- private final StatsLogger perLogStatsLogger;
-
- protected final AtomicBoolean closed = new AtomicBoolean(false);
-
- public BKDistributedLogNamespace(
- DistributedLogConfiguration conf,
- URI uri,
- NamespaceDriver driver,
- OrderedScheduler scheduler,
- FeatureProvider featureProvider,
- PermitLimiter writeLimiter,
- AsyncFailureInjector failureInjector,
- StatsLogger statsLogger,
- StatsLogger perLogStatsLogger,
- String clientId,
- 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;
-
- // create a log segment metadata cache
- this.logSegmentMetadataCache = new LogSegmentMetadataCache(conf, Ticker.systemTicker());
- }
-
- @Override
- public NamespaceDriver getNamespaceDriver() {
- return driver;
- }
-
- //
- // Namespace Methods
- //
-
- @Override
- public void createLog(String logName)
- throws InvalidStreamNameException, IOException {
- checkState();
- validateName(logName);
- URI uri = FutureUtils.result(driver.getLogMetadataStore().createLog(logName));
- FutureUtils.result(driver.getLogStreamMetadataStore(WRITER).getLog(uri, logName, true, true));
- }
-
- @Override
- public void deleteLog(String logName)
- throws InvalidStreamNameException, LogNotFoundException, IOException {
- checkState();
- validateName(logName);
- Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
- if (!uri.isPresent()) {
- throw new LogNotFoundException("Log " + logName + " isn't found.");
- }
- DistributedLogManager dlm = openLogInternal(
- uri.get(),
- logName,
- Optional.<DistributedLogConfiguration>absent(),
- Optional.<DynamicDistributedLogConfiguration>absent());
- dlm.delete();
- }
-
- @Override
- public DistributedLogManager openLog(String logName)
- throws InvalidStreamNameException, IOException {
- return openLog(logName,
- Optional.<DistributedLogConfiguration>absent(),
- Optional.<DynamicDistributedLogConfiguration>absent(),
- Optional.<StatsLogger>absent());
- }
-
- @Override
- public DistributedLogManager openLog(String logName,
- Optional<DistributedLogConfiguration> logConf,
- Optional<DynamicDistributedLogConfiguration> dynamicLogConf,
- Optional<StatsLogger> perStreamStatsLogger)
- throws InvalidStreamNameException, IOException {
- checkState();
- validateName(logName);
- Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
- if (!uri.isPresent()) {
- throw new LogNotFoundException("Log " + logName + " isn't found.");
- }
- return openLogInternal(
- uri.get(),
- logName,
- logConf,
- dynamicLogConf);
- }
-
- @Override
- public boolean logExists(String logName)
- throws IOException, IllegalArgumentException {
- checkState();
- Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
- if (uri.isPresent()) {
- try {
- FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
- .logExists(uri.get(), logName));
- return true;
- } catch (LogNotFoundException lnfe) {
- return false;
- }
- } else {
- return false;
- }
- }
-
- @Override
- public Iterator<String> getLogs() throws IOException {
- checkState();
- return FutureUtils.result(driver.getLogMetadataStore().getLogs());
- }
-
- @Override
- public void registerNamespaceListener(NamespaceListener listener) {
- driver.getLogMetadataStore().registerNamespaceListener(listener);
- }
-
- @Override
- public synchronized AccessControlManager createAccessControlManager() throws IOException {
- checkState();
- return driver.getAccessControlManager();
- }
-
- /**
- * Open the log in location <i>uri</i>.
- *
- * @param uri
- * location to store the log
- * @param nameOfLogStream
- * name of the log
- * @param logConfiguration
- * optional stream configuration
- * @param dynamicLogConfiguration
- * dynamic stream configuration overrides.
- * @return distributedlog manager instance.
- * @throws InvalidStreamNameException if the stream name is invalid
- * @throws IOException
- */
- protected DistributedLogManager openLogInternal(
- URI uri,
- String nameOfLogStream,
- Optional<DistributedLogConfiguration> logConfiguration,
- Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration)
- throws InvalidStreamNameException, IOException {
- // Make sure the name is well formed
- checkState();
- validateName(nameOfLogStream);
-
- DistributedLogConfiguration mergedConfiguration = new DistributedLogConfiguration();
- mergedConfiguration.addConfiguration(conf);
- mergedConfiguration.loadStreamConf(logConfiguration);
- // If dynamic config was not provided, default to a static view of the global configuration.
- DynamicDistributedLogConfiguration dynConf = null;
- if (dynamicLogConfiguration.isPresent()) {
- dynConf = dynamicLogConfiguration.get();
- } else {
- dynConf = ConfUtils.getConstDynConf(mergedConfiguration);
- }
-
- return new BKDistributedLogManager(
- nameOfLogStream, /* Log Name */
- mergedConfiguration, /* Configuration */
- dynConf, /* Dynamic Configuration */
- uri, /* Namespace URI */
- driver, /* Namespace Driver */
- logSegmentMetadataCache, /* Log Segment Metadata Cache */
- scheduler, /* DL scheduler */
- clientId, /* Client Id */
- regionId, /* Region Id */
- writeLimiter, /* Write Limiter */
- featureProvider.scope("dl"), /* Feature Provider */
- failureInjector, /* Failure Injector */
- statsLogger, /* Stats Logger */
- perLogStatsLogger, /* Per Log Stats Logger */
- Optional.<AsyncCloseable>absent() /* shared resources, we don't need to close any resources in dlm */
- );
- }
-
- /**
- * Check the namespace state.
- *
- * @throws IOException
- */
- private void checkState() throws IOException {
- if (closed.get()) {
- LOG.error("BK namespace {} is already closed", namespace);
- throw new AlreadyClosedException("BK namespace " + namespace + " is already closed");
- }
- }
-
- /**
- * Close the distributed log manager factory, freeing any resources it may hold.
- */
- @Override
- public void close() {
- if (!closed.compareAndSet(false, true)) {
- return;
- }
- // shutdown the driver
- Utils.close(driver);
- // close the write limiter
- this.writeLimiter.close();
- // Shutdown the schedulers
- SchedulerUtils.shutdownScheduler(scheduler, conf.getSchedulerShutdownTimeoutMs(),
- TimeUnit.MILLISECONDS);
- LOG.info("Executor Service Stopped.");
- }
-}
[42/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java
new file mode 100644
index 0000000..666fa31
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java
@@ -0,0 +1,500 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Maps;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.service.DLSocketAddress;
+import com.twitter.finagle.ChannelException;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.Counter;
+import com.twitter.finagle.stats.Gauge;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.util.Function0;
+import java.net.SocketAddress;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.commons.lang3.tuple.Pair;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.Seq;
+
+/**
+ * Consistent Hashing Based {@link RoutingService}.
+ */
+public class ConsistentHashRoutingService extends ServerSetRoutingService {
+
+ private static final Logger logger = LoggerFactory.getLogger(ConsistentHashRoutingService.class);
+
+ @Deprecated
+ public static ConsistentHashRoutingService of(ServerSetWatcher serverSetWatcher, int numReplicas) {
+ return new ConsistentHashRoutingService(serverSetWatcher, numReplicas, 300, NullStatsReceiver.get());
+ }
+
+ /**
+ * Builder helper class to build a consistent hash bashed {@link RoutingService}.
+ *
+ * @return builder to build a consistent hash based {@link RoutingService}.
+ */
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ /**
+ * Builder for building consistent hash based routing service.
+ */
+ public static class Builder implements RoutingService.Builder {
+
+ private ServerSet serverSet;
+ private boolean resolveFromName = false;
+ private int numReplicas;
+ private int blackoutSeconds = 300;
+ private StatsReceiver statsReceiver = NullStatsReceiver.get();
+
+ private Builder() {}
+
+ public Builder serverSet(ServerSet serverSet) {
+ this.serverSet = serverSet;
+ return this;
+ }
+
+ public Builder resolveFromName(boolean enabled) {
+ this.resolveFromName = enabled;
+ return this;
+ }
+
+ public Builder numReplicas(int numReplicas) {
+ this.numReplicas = numReplicas;
+ return this;
+ }
+
+ public Builder blackoutSeconds(int seconds) {
+ this.blackoutSeconds = seconds;
+ return this;
+ }
+
+ public Builder statsReceiver(StatsReceiver statsReceiver) {
+ this.statsReceiver = statsReceiver;
+ return this;
+ }
+
+ @Override
+ public RoutingService build() {
+ checkNotNull(serverSet, "No serverset provided.");
+ checkNotNull(statsReceiver, "No stats receiver provided.");
+ checkArgument(numReplicas > 0, "Invalid number of replicas : " + numReplicas);
+ return new ConsistentHashRoutingService(new TwitterServerSetWatcher(serverSet, resolveFromName),
+ numReplicas, blackoutSeconds, statsReceiver);
+ }
+ }
+
+ static class ConsistentHash {
+ private final HashFunction hashFunction;
+ private final int numOfReplicas;
+ private final SortedMap<Long, SocketAddress> circle;
+
+ // Stats
+ protected final Counter hostAddedCounter;
+ protected final Counter hostRemovedCounter;
+
+ ConsistentHash(HashFunction hashFunction,
+ int numOfReplicas,
+ StatsReceiver statsReceiver) {
+ this.hashFunction = hashFunction;
+ this.numOfReplicas = numOfReplicas;
+ this.circle = new TreeMap<Long, SocketAddress>();
+
+ this.hostAddedCounter = statsReceiver.counter0("adds");
+ this.hostRemovedCounter = statsReceiver.counter0("removes");
+ }
+
+ private String replicaName(int shardId, int replica, String address) {
+ if (shardId < 0) {
+ shardId = UNKNOWN_SHARD_ID;
+ }
+
+ StringBuilder sb = new StringBuilder(100);
+ sb.append("shard-");
+ sb.append(shardId);
+ sb.append('-');
+ sb.append(replica);
+ sb.append('-');
+ sb.append(address);
+
+ return sb.toString();
+ }
+
+ private Long replicaHash(int shardId, int replica, String address) {
+ return hashFunction.hashUnencodedChars(replicaName(shardId, replica, address)).asLong();
+ }
+
+ private Long replicaHash(int shardId, int replica, SocketAddress address) {
+ return replicaHash(shardId, replica, address.toString());
+ }
+
+ public synchronized void add(int shardId, SocketAddress address) {
+ String addressStr = address.toString();
+ for (int i = 0; i < numOfReplicas; i++) {
+ Long hash = replicaHash(shardId, i, addressStr);
+ circle.put(hash, address);
+ }
+ hostAddedCounter.incr();
+ }
+
+ public synchronized void remove(int shardId, SocketAddress address) {
+ for (int i = 0; i < numOfReplicas; i++) {
+ long hash = replicaHash(shardId, i, address);
+ SocketAddress oldAddress = circle.get(hash);
+ if (null != oldAddress && oldAddress.equals(address)) {
+ circle.remove(hash);
+ }
+ }
+ hostRemovedCounter.incr();
+ }
+
+ public SocketAddress get(String key, RoutingContext rContext) {
+ long hash = hashFunction.hashUnencodedChars(key).asLong();
+ return find(hash, rContext);
+ }
+
+ private synchronized SocketAddress find(long hash, RoutingContext rContext) {
+ if (circle.isEmpty()) {
+ return null;
+ }
+
+ Iterator<Map.Entry<Long, SocketAddress>> iterator =
+ circle.tailMap(hash).entrySet().iterator();
+ while (iterator.hasNext()) {
+ Map.Entry<Long, SocketAddress> entry = iterator.next();
+ if (!rContext.isTriedHost(entry.getValue())) {
+ return entry.getValue();
+ }
+ }
+ // the tail map has been checked
+ iterator = circle.headMap(hash).entrySet().iterator();
+ while (iterator.hasNext()) {
+ Map.Entry<Long, SocketAddress> entry = iterator.next();
+ if (!rContext.isTriedHost(entry.getValue())) {
+ return entry.getValue();
+ }
+ }
+
+ return null;
+ }
+
+ private synchronized Pair<Long, SocketAddress> get(long hash) {
+ if (circle.isEmpty()) {
+ return null;
+ }
+
+ if (!circle.containsKey(hash)) {
+ SortedMap<Long, SocketAddress> tailMap = circle.tailMap(hash);
+ hash = tailMap.isEmpty() ? circle.firstKey() : tailMap.firstKey();
+ }
+ return Pair.of(hash, circle.get(hash));
+ }
+
+ synchronized void dumpHashRing() {
+ for (Map.Entry<Long, SocketAddress> entry : circle.entrySet()) {
+ logger.info(entry.getKey() + " : " + entry.getValue());
+ }
+ }
+
+ }
+
+ class BlackoutHost implements TimerTask {
+ final int shardId;
+ final SocketAddress address;
+
+ BlackoutHost(int shardId, SocketAddress address) {
+ this.shardId = shardId;
+ this.address = address;
+ numBlackoutHosts.incrementAndGet();
+ }
+
+ @Override
+ public void run(Timeout timeout) throws Exception {
+ numBlackoutHosts.decrementAndGet();
+ if (!timeout.isExpired()) {
+ return;
+ }
+ Set<SocketAddress> removedList = new HashSet<SocketAddress>();
+ boolean joined;
+ // add the shard back
+ synchronized (shardId2Address) {
+ SocketAddress curHost = shardId2Address.get(shardId);
+ if (null != curHost) {
+ // there is already new shard joint, so drop the host.
+ logger.info("Blackout Shard {} ({}) was already replaced by {} permanently.",
+ new Object[] { shardId, address, curHost });
+ joined = false;
+ } else {
+ join(shardId, address, removedList);
+ joined = true;
+ }
+ }
+ if (joined) {
+ for (RoutingListener listener : listeners) {
+ listener.onServerJoin(address);
+ }
+ } else {
+ for (RoutingListener listener : listeners) {
+ listener.onServerLeft(address);
+ }
+ }
+ }
+ }
+
+ protected final HashedWheelTimer hashedWheelTimer;
+ protected final HashFunction hashFunction = Hashing.md5();
+ protected final ConsistentHash circle;
+ protected final Map<Integer, SocketAddress> shardId2Address =
+ new HashMap<Integer, SocketAddress>();
+ protected final Map<SocketAddress, Integer> address2ShardId =
+ new HashMap<SocketAddress, Integer>();
+
+ // blackout period
+ protected final int blackoutSeconds;
+
+ // stats
+ protected final StatsReceiver statsReceiver;
+ protected final AtomicInteger numBlackoutHosts;
+ protected final Gauge numBlackoutHostsGauge;
+ protected final Gauge numHostsGauge;
+
+ private static final int UNKNOWN_SHARD_ID = -1;
+
+ ConsistentHashRoutingService(ServerSetWatcher serverSetWatcher,
+ int numReplicas,
+ int blackoutSeconds,
+ StatsReceiver statsReceiver) {
+ super(serverSetWatcher);
+ this.circle = new ConsistentHash(hashFunction, numReplicas, statsReceiver.scope("ring"));
+ this.hashedWheelTimer = new HashedWheelTimer(new ThreadFactoryBuilder()
+ .setNameFormat("ConsistentHashRoutingService-Timer-%d").build());
+ this.blackoutSeconds = blackoutSeconds;
+ // stats
+ this.statsReceiver = statsReceiver;
+ this.numBlackoutHosts = new AtomicInteger(0);
+ this.numBlackoutHostsGauge = this.statsReceiver.addGauge(gaugeName("num_blackout_hosts"),
+ new Function0<Object>() {
+ @Override
+ public Object apply() {
+ return (float) numBlackoutHosts.get();
+ }
+ });
+ this.numHostsGauge = this.statsReceiver.addGauge(gaugeName("num_hosts"),
+ new Function0<Object>() {
+ @Override
+ public Object apply() {
+ return (float) address2ShardId.size();
+ }
+ });
+ }
+
+ private static Seq<String> gaugeName(String name) {
+ return scala.collection.JavaConversions.asScalaBuffer(Arrays.asList(name)).toList();
+ }
+
+ @Override
+ public void startService() {
+ super.startService();
+ this.hashedWheelTimer.start();
+ }
+
+ @Override
+ public void stopService() {
+ this.hashedWheelTimer.stop();
+ super.stopService();
+ }
+
+ @Override
+ public Set<SocketAddress> getHosts() {
+ synchronized (shardId2Address) {
+ return ImmutableSet.copyOf(address2ShardId.keySet());
+ }
+ }
+
+ @Override
+ public SocketAddress getHost(String key, RoutingContext rContext)
+ throws NoBrokersAvailableException {
+ SocketAddress host = circle.get(key, rContext);
+ if (null != host) {
+ return host;
+ }
+ throw new NoBrokersAvailableException("No host found for " + key + ", routing context : " + rContext);
+ }
+
+ @Override
+ public void removeHost(SocketAddress host, Throwable reason) {
+ removeHostInternal(host, Optional.of(reason));
+ }
+
+ private void removeHostInternal(SocketAddress host, Optional<Throwable> reason) {
+ synchronized (shardId2Address) {
+ Integer shardId = address2ShardId.remove(host);
+ if (null != shardId) {
+ SocketAddress curHost = shardId2Address.get(shardId);
+ if (null != curHost && curHost.equals(host)) {
+ shardId2Address.remove(shardId);
+ }
+ circle.remove(shardId, host);
+ if (reason.isPresent()) {
+ if (reason.get() instanceof ChannelException) {
+ logger.info("Shard {} ({}) left due to ChannelException, black it out for {} seconds"
+ + " (message = {})",
+ new Object[] { shardId, host, blackoutSeconds, reason.get().toString() });
+ BlackoutHost blackoutHost = new BlackoutHost(shardId, host);
+ hashedWheelTimer.newTimeout(blackoutHost, blackoutSeconds, TimeUnit.SECONDS);
+ } else {
+ logger.info("Shard {} ({}) left due to exception {}",
+ new Object[] { shardId, host, reason.get().toString() });
+ }
+ } else {
+ logger.info("Shard {} ({}) left after server set change",
+ shardId, host);
+ }
+ } else if (reason.isPresent()) {
+ logger.info("Node {} left due to exception {}", host, reason.get().toString());
+ } else {
+ logger.info("Node {} left after server set change", host);
+ }
+ }
+ }
+
+ /**
+ * The caller should synchronize on <i>shardId2Address</i>.
+ * @param shardId
+ * Shard id of new host joined.
+ * @param newHost
+ * New host joined.
+ * @param removedList
+ * Old hosts to remove
+ */
+ private void join(int shardId, SocketAddress newHost, Set<SocketAddress> removedList) {
+ SocketAddress oldHost = shardId2Address.put(shardId, newHost);
+ if (null != oldHost) {
+ // remove the old host only when a new shard is kicked in to replace it.
+ address2ShardId.remove(oldHost);
+ circle.remove(shardId, oldHost);
+ removedList.add(oldHost);
+ logger.info("Shard {} ({}) left permanently.", shardId, oldHost);
+ }
+ address2ShardId.put(newHost, shardId);
+ circle.add(shardId, newHost);
+ logger.info("Shard {} ({}) joined to replace ({}).",
+ new Object[] { shardId, newHost, oldHost });
+ }
+
+ @Override
+ protected synchronized void performServerSetChange(ImmutableSet<DLSocketAddress> serviceInstances) {
+ Set<SocketAddress> joinedList = new HashSet<SocketAddress>();
+ Set<SocketAddress> removedList = new HashSet<SocketAddress>();
+
+ Map<Integer, SocketAddress> newMap = new HashMap<Integer, SocketAddress>();
+ synchronized (shardId2Address) {
+ for (DLSocketAddress serviceInstance : serviceInstances) {
+ if (serviceInstance.getShard() >= 0) {
+ newMap.put(serviceInstance.getShard(), serviceInstance.getSocketAddress());
+ } else {
+ Integer shard = address2ShardId.get(serviceInstance.getSocketAddress());
+ if (null == shard) {
+ // Assign a random negative shardId
+ int shardId;
+ do {
+ shardId = Math.min(-1 , (int) (Math.random() * Integer.MIN_VALUE));
+ } while (null != shardId2Address.get(shardId));
+ shard = shardId;
+ }
+ newMap.put(shard, serviceInstance.getSocketAddress());
+ }
+ }
+ }
+
+ Map<Integer, SocketAddress> left;
+ synchronized (shardId2Address) {
+ MapDifference<Integer, SocketAddress> difference =
+ Maps.difference(shardId2Address, newMap);
+ left = difference.entriesOnlyOnLeft();
+ for (Map.Entry<Integer, SocketAddress> shardEntry : left.entrySet()) {
+ int shard = shardEntry.getKey();
+ if (shard >= 0) {
+ SocketAddress host = shardId2Address.get(shard);
+ if (null != host) {
+ // we don't remove those hosts that just disappered on serverset proactively,
+ // since it might be just because serverset become flaky
+ // address2ShardId.remove(host);
+ // circle.remove(shard, host);
+ logger.info("Shard {} ({}) left temporarily.", shard, host);
+ }
+ } else {
+ // shard id is negative - they are resolved from finagle name, which instances don't have shard id
+ // in this case, if they are removed from serverset, we removed them directly
+ SocketAddress host = shardEntry.getValue();
+ if (null != host) {
+ removeHostInternal(host, Optional.<Throwable>absent());
+ removedList.add(host);
+ }
+ }
+ }
+ // we need to find if any shards are replacing old shards
+ for (Map.Entry<Integer, SocketAddress> shard : newMap.entrySet()) {
+ SocketAddress oldHost = shardId2Address.get(shard.getKey());
+ SocketAddress newHost = shard.getValue();
+ if (!newHost.equals(oldHost)) {
+ join(shard.getKey(), newHost, removedList);
+ joinedList.add(newHost);
+ }
+ }
+ }
+
+ for (SocketAddress addr : removedList) {
+ for (RoutingListener listener : listeners) {
+ listener.onServerLeft(addr);
+ }
+ }
+
+ for (SocketAddress addr : joinedList) {
+ for (RoutingListener listener : listeners) {
+ listener.onServerJoin(addr);
+ }
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java
new file mode 100644
index 0000000..e51eb1e
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java
@@ -0,0 +1,263 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import com.google.common.collect.ImmutableSet;
+import com.twitter.common.base.Command;
+import com.twitter.common.base.Commands;
+import com.twitter.common.zookeeper.Group;
+import com.twitter.common.zookeeper.ServerSet;
+import com.twitter.finagle.Addr;
+import com.twitter.finagle.Address;
+import com.twitter.finagle.Name;
+import com.twitter.finagle.Resolver$;
+import com.twitter.thrift.Endpoint;
+import com.twitter.thrift.ServiceInstance;
+import com.twitter.thrift.Status;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Finagle Name based {@link ServerSet} implementation.
+ */
+class NameServerSet implements ServerSet {
+
+ private static final Logger logger = LoggerFactory.getLogger(NameServerSet.class);
+
+ private volatile Set<HostChangeMonitor<ServiceInstance>> watchers =
+ new HashSet<HostChangeMonitor<ServiceInstance>>();
+ private volatile ImmutableSet<ServiceInstance> hostSet = ImmutableSet.of();
+ private AtomicBoolean resolutionPending = new AtomicBoolean(true);
+
+ public NameServerSet(String nameStr) {
+ Name name;
+ try {
+ name = Resolver$.MODULE$.eval(nameStr);
+ } catch (Exception exc) {
+ logger.error("Exception in Resolver.eval for name {}", nameStr, exc);
+ // Since this is called from various places that dont handle specific exceptions,
+ // we have no option than to throw a runtime exception to halt the control flow
+ // This should only happen in case of incorrect configuration. Having a log message
+ // would help identify the problem during tests
+ throw new RuntimeException(exc);
+ }
+ initialize(name);
+ }
+
+ public NameServerSet(Name name) {
+ initialize(name);
+ }
+
+ private void initialize(Name name) {
+ if (name instanceof TestName) {
+ ((TestName) name).changes(new AbstractFunction1<Addr, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(Addr varAddr) {
+ return NameServerSet.this.respondToChanges(varAddr);
+ }
+ });
+ } else if (name instanceof Name.Bound) {
+ ((Name.Bound) name).addr().changes().respond(new AbstractFunction1<Addr, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(Addr varAddr) {
+ return NameServerSet.this.respondToChanges(varAddr);
+ }
+ });
+ } else {
+ logger.error("NameServerSet only supports Name.Bound. While the resolved name {} was {}",
+ name, name.getClass());
+ throw new UnsupportedOperationException("NameServerSet only supports Name.Bound");
+ }
+ }
+
+ private ServiceInstance endpointAddressToServiceInstance(Address endpointAddress) {
+ if (endpointAddress instanceof Address.Inet) {
+ InetSocketAddress inetSocketAddress = ((Address.Inet) endpointAddress).addr();
+ Endpoint endpoint = new Endpoint(inetSocketAddress.getHostString(), inetSocketAddress.getPort());
+ HashMap<String, Endpoint> map = new HashMap<String, Endpoint>();
+ map.put("thrift", endpoint);
+ return new ServiceInstance(
+ endpoint,
+ map,
+ Status.ALIVE);
+ } else {
+ logger.error("We expect InetSocketAddress while the resolved address {} was {}",
+ endpointAddress, endpointAddress.getClass());
+ throw new UnsupportedOperationException("invalid endpoint address: " + endpointAddress);
+ }
+ }
+
+
+ private BoxedUnit respondToChanges(Addr addr) {
+ ImmutableSet<ServiceInstance> oldHostSet = ImmutableSet.copyOf(hostSet);
+
+ ImmutableSet<ServiceInstance> newHostSet = oldHostSet;
+
+ if (addr instanceof Addr.Bound) {
+ scala.collection.immutable.Set<Address> endpointAddresses = ((Addr.Bound) addr).addrs();
+ scala.collection.Iterator<Address> endpointAddressesIterator = endpointAddresses.toIterator();
+ HashSet<ServiceInstance> serviceInstances = new HashSet<ServiceInstance>();
+ while (endpointAddressesIterator.hasNext()) {
+ serviceInstances.add(endpointAddressToServiceInstance(endpointAddressesIterator.next()));
+ }
+ newHostSet = ImmutableSet.copyOf(serviceInstances);
+
+ } else if (addr instanceof Addr.Failed) {
+ logger.error("Name resolution failed", ((Addr.Failed) addr).cause());
+ newHostSet = ImmutableSet.of();
+ } else if (addr.toString().equals("Pending")) {
+ logger.info("Name resolution pending");
+ newHostSet = oldHostSet;
+ } else if (addr.toString().equals("Neg")) {
+ newHostSet = ImmutableSet.of();
+ } else {
+ logger.error("Invalid Addr type: {}", addr.getClass().getName());
+ throw new UnsupportedOperationException("Invalid Addr type:" + addr.getClass().getName());
+ }
+
+ // Reference comparison is valid as the sets are immutable
+ if (oldHostSet != newHostSet) {
+ logger.info("NameServerSet updated: {} -> {}", hostSetToString(oldHostSet), hostSetToString(newHostSet));
+ resolutionPending.set(false);
+ hostSet = newHostSet;
+ synchronized (watchers) {
+ for (HostChangeMonitor<ServiceInstance> watcher: watchers) {
+ watcher.onChange(newHostSet);
+ }
+ }
+
+ }
+
+ return BoxedUnit.UNIT;
+ }
+
+
+ private String hostSetToString(ImmutableSet<ServiceInstance> hostSet) {
+ StringBuilder result = new StringBuilder();
+ result.append("(");
+ for (ServiceInstance serviceInstance : hostSet) {
+ Endpoint endpoint = serviceInstance.getServiceEndpoint();
+ result.append(String.format(" %s:%d", endpoint.getHost(), endpoint.getPort()));
+ }
+ result.append(" )");
+
+ return result.toString();
+ }
+
+
+ /**
+ * Attempts to join a server set for this logical service group.
+ *
+ * @param endpoint the primary service endpoint
+ * @param additionalEndpoints and additional endpoints keyed by their logical name
+ * @param status the current service status
+ * @return an EndpointStatus object that allows the endpoint to adjust its status
+ * @throws Group.JoinException if there was a problem joining the server set
+ * @throws InterruptedException if interrupted while waiting to join the server set
+ * @deprecated The status field is deprecated. Please use {@link #join(java.net.InetSocketAddress, java.util.Map)}
+ */
+ @Override
+ public EndpointStatus join(InetSocketAddress endpoint,
+ Map<String, InetSocketAddress> additionalEndpoints,
+ Status status)
+ throws Group.JoinException, InterruptedException {
+ throw new UnsupportedOperationException("NameServerSet does not support join");
+ }
+
+ /**
+ * Attempts to join a server set for this logical service group.
+ *
+ * @param endpoint the primary service endpoint
+ * @param additionalEndpoints and additional endpoints keyed by their logical name
+ * @return an EndpointStatus object that allows the endpoint to adjust its status
+ * @throws Group.JoinException if there was a problem joining the server set
+ * @throws InterruptedException if interrupted while waiting to join the server set
+ */
+ @Override
+ public EndpointStatus join(InetSocketAddress endpoint, Map<String, InetSocketAddress> additionalEndpoints)
+ throws Group.JoinException, InterruptedException {
+ throw new UnsupportedOperationException("NameServerSet does not support join");
+ }
+
+ /**
+ * Attempts to join a server set for this logical service group.
+ *
+ * @param endpoint the primary service endpoint
+ * @param additionalEndpoints and additional endpoints keyed by their logical name
+ * @param shardId Unique shard identifier for this member of the service.
+ * @return an EndpointStatus object that allows the endpoint to adjust its status
+ * @throws Group.JoinException if there was a problem joining the server set
+ * @throws InterruptedException if interrupted while waiting to join the server set
+ */
+ @Override
+ public EndpointStatus join(InetSocketAddress endpoint,
+ Map<String, InetSocketAddress> additionalEndpoints,
+ int shardId)
+ throws Group.JoinException, InterruptedException {
+ throw new UnsupportedOperationException("NameServerSet does not support join");
+ }
+
+ /**
+ * Registers a monitor to receive change notices for this server set as long as this jvm process
+ * is alive. Blocks until the initial server set can be gathered and delivered to the monitor.
+ * The monitor will be notified if the membership set or parameters of existing members have
+ * changed.
+ *
+ * @param monitor the server set monitor to call back when the host set changes
+ * @throws com.twitter.common.net.pool.DynamicHostSet.MonitorException if there is a problem monitoring the host set
+ * @deprecated Deprecated in favor of {@link #watch(com.twitter.common.net.pool.DynamicHostSet.HostChangeMonitor)}
+ */
+ @Deprecated
+ @Override
+ public void monitor(HostChangeMonitor<ServiceInstance> monitor) throws MonitorException {
+ throw new UnsupportedOperationException("NameServerSet does not support monitor");
+ }
+
+ /**
+ * Registers a monitor to receive change notices for this server set as long as this jvm process
+ * is alive. Blocks until the initial server set can be gathered and delivered to the monitor.
+ * The monitor will be notified if the membership set or parameters of existing members have
+ * changed.
+ *
+ * @param monitor the server set monitor to call back when the host set changes
+ * @return A command which, when executed, will stop monitoring the host set.
+ * @throws com.twitter.common.net.pool.DynamicHostSet.MonitorException if there is a problem monitoring the host set
+ */
+ @Override
+ public Command watch(HostChangeMonitor<ServiceInstance> monitor) throws MonitorException {
+ // First add the monitor to the watchers so that it does not miss any changes and invoke
+ // the onChange method
+ synchronized (watchers) {
+ watchers.add(monitor);
+ }
+
+ if (resolutionPending.compareAndSet(false, false)) {
+ monitor.onChange(hostSet);
+ }
+
+ return Commands.NOOP; // Return value is not used
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java
new file mode 100644
index 0000000..d71cee3
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java
@@ -0,0 +1,192 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.Set;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Chain multiple routing services.
+ */
+public class RegionsRoutingService implements RoutingService {
+
+ private static final Logger logger = LoggerFactory.getLogger(RegionsRoutingService.class);
+
+ /**
+ * Create a multiple regions routing services based on a list of region routing {@code services}.
+ *
+ * <p>It is deprecated. Please use {@link Builder} to build multiple regions routing service.
+ *
+ * @param regionResolver region resolver
+ * @param services a list of region routing services.
+ * @return multiple regions routing service
+ * @see Builder
+ */
+ @Deprecated
+ public static RegionsRoutingService of(RegionResolver regionResolver,
+ RoutingService...services) {
+ return new RegionsRoutingService(regionResolver, services);
+ }
+
+ /**
+ * Create a builder to build a multiple-regions routing service.
+ *
+ * @return builder to build a multiple-regions routing service.
+ */
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ /**
+ * Builder to build a multiple-regions routing service.
+ */
+ public static class Builder implements RoutingService.Builder {
+
+ private RegionResolver resolver;
+ private RoutingService.Builder[] routingServiceBuilders;
+ private StatsReceiver statsReceiver = NullStatsReceiver.get();
+
+ private Builder() {}
+
+ public Builder routingServiceBuilders(RoutingService.Builder...builders) {
+ this.routingServiceBuilders = builders;
+ return this;
+ }
+
+ public Builder resolver(RegionResolver regionResolver) {
+ this.resolver = regionResolver;
+ return this;
+ }
+
+ @Override
+ public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
+ this.statsReceiver = statsReceiver;
+ return this;
+ }
+
+ @Override
+ public RegionsRoutingService build() {
+ checkNotNull(routingServiceBuilders, "No routing service builder provided.");
+ checkNotNull(resolver, "No region resolver provided.");
+ checkNotNull(statsReceiver, "No stats receiver provided");
+ RoutingService[] services = new RoutingService[routingServiceBuilders.length];
+ for (int i = 0; i < services.length; i++) {
+ String statsScope;
+ if (0 == i) {
+ statsScope = "local";
+ } else {
+ statsScope = "remote_" + i;
+ }
+ services[i] = routingServiceBuilders[i]
+ .statsReceiver(statsReceiver.scope(statsScope))
+ .build();
+ }
+ return new RegionsRoutingService(resolver, services);
+ }
+ }
+
+ protected final RegionResolver regionResolver;
+ protected final RoutingService[] routingServices;
+
+ private RegionsRoutingService(RegionResolver resolver,
+ RoutingService[] routingServices) {
+ this.regionResolver = resolver;
+ this.routingServices = routingServices;
+ }
+
+ @Override
+ public Set<SocketAddress> getHosts() {
+ Set<SocketAddress> hosts = Sets.newHashSet();
+ for (RoutingService rs : routingServices) {
+ hosts.addAll(rs.getHosts());
+ }
+ return hosts;
+ }
+
+ @Override
+ public void startService() {
+ for (RoutingService service : routingServices) {
+ service.startService();
+ }
+ logger.info("Regions Routing Service Started");
+ }
+
+ @Override
+ public void stopService() {
+ for (RoutingService service : routingServices) {
+ service.stopService();
+ }
+ logger.info("Regions Routing Service Stopped");
+ }
+
+ @Override
+ public RoutingService registerListener(RoutingListener listener) {
+ for (RoutingService service : routingServices) {
+ service.registerListener(listener);
+ }
+ return this;
+ }
+
+ @Override
+ public RoutingService unregisterListener(RoutingListener listener) {
+ for (RoutingService service : routingServices) {
+ service.registerListener(listener);
+ }
+ return this;
+ }
+
+ @Override
+ public SocketAddress getHost(String key, RoutingContext routingContext)
+ throws NoBrokersAvailableException {
+ for (RoutingService service : routingServices) {
+ try {
+ SocketAddress addr = service.getHost(key, routingContext);
+ if (routingContext.hasUnavailableRegions()) {
+ // current region is unavailable
+ String region = regionResolver.resolveRegion(addr);
+ if (routingContext.isUnavailableRegion(region)) {
+ continue;
+ }
+ }
+ if (!routingContext.isTriedHost(addr)) {
+ return addr;
+ }
+ } catch (NoBrokersAvailableException nbae) {
+ // if there isn't broker available in current service, try next service.
+ logger.debug("No brokers available in region {} : ", service, nbae);
+ }
+ }
+ throw new NoBrokersAvailableException("No host found for " + key + ", routing context : " + routingContext);
+ }
+
+ @Override
+ public void removeHost(SocketAddress address, Throwable reason) {
+ for (RoutingService service : routingServices) {
+ service.removeHost(address, reason);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java
new file mode 100644
index 0000000..ad73c17
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java
@@ -0,0 +1,206 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Routing Service provides mechanism how to route requests.
+ */
+public interface RoutingService {
+
+ /**
+ * Builder to build routing service.
+ */
+ interface Builder {
+
+ /**
+ * Build routing service with stats receiver.
+ *
+ * @param statsReceiver
+ * stats receiver
+ * @return built routing service
+ */
+ Builder statsReceiver(StatsReceiver statsReceiver);
+
+ /**
+ * Build the routing service.
+ *
+ * @return built routing service
+ */
+ RoutingService build();
+
+ }
+
+ /**
+ * Listener for server changes on routing service.
+ */
+ interface RoutingListener {
+ /**
+ * Trigger when server left.
+ *
+ * @param address left server.
+ */
+ void onServerLeft(SocketAddress address);
+
+ /**
+ * Trigger when server joint.
+ *
+ * @param address joint server.
+ */
+ void onServerJoin(SocketAddress address);
+ }
+
+ /**
+ * Routing Context of a request.
+ */
+ class RoutingContext {
+
+ public static RoutingContext of(RegionResolver resolver) {
+ return new RoutingContext(resolver);
+ }
+
+ final RegionResolver regionResolver;
+ final Map<SocketAddress, StatusCode> triedHosts;
+ final Set<String> unavailableRegions;
+
+ private RoutingContext(RegionResolver regionResolver) {
+ this.regionResolver = regionResolver;
+ this.triedHosts = new HashMap<SocketAddress, StatusCode>();
+ this.unavailableRegions = new HashSet<String>();
+ }
+
+ @Override
+ public synchronized String toString() {
+ return "(tried hosts=" + triedHosts + ")";
+ }
+
+ /**
+ * Add tried host to routing context.
+ *
+ * @param socketAddress
+ * socket address of tried host.
+ * @param code
+ * status code returned from tried host.
+ * @return routing context.
+ */
+ public synchronized RoutingContext addTriedHost(SocketAddress socketAddress, StatusCode code) {
+ this.triedHosts.put(socketAddress, code);
+ if (StatusCode.REGION_UNAVAILABLE == code) {
+ unavailableRegions.add(regionResolver.resolveRegion(socketAddress));
+ }
+ return this;
+ }
+
+ /**
+ * Is the host <i>address</i> already tried.
+ *
+ * @param address
+ * socket address to check
+ * @return true if the address is already tried, otherwise false.
+ */
+ public synchronized boolean isTriedHost(SocketAddress address) {
+ return this.triedHosts.containsKey(address);
+ }
+
+ /**
+ * Whether encountered unavailable regions.
+ *
+ * @return true if encountered unavailable regions, otherwise false.
+ */
+ public synchronized boolean hasUnavailableRegions() {
+ return !unavailableRegions.isEmpty();
+ }
+
+ /**
+ * Whether the <i>region</i> is unavailable.
+ *
+ * @param region
+ * region
+ * @return true if the region is unavailable, otherwise false.
+ */
+ public synchronized boolean isUnavailableRegion(String region) {
+ return unavailableRegions.contains(region);
+ }
+
+ }
+
+ /**
+ * Start routing service.
+ */
+ void startService();
+
+ /**
+ * Stop routing service.
+ */
+ void stopService();
+
+ /**
+ * Register routing listener.
+ *
+ * @param listener routing listener.
+ * @return routing service.
+ */
+ RoutingService registerListener(RoutingListener listener);
+
+ /**
+ * Unregister routing listener.
+ *
+ * @param listener routing listener.
+ * @return routing service.
+ */
+ RoutingService unregisterListener(RoutingListener listener);
+
+ /**
+ * Get all the hosts that available in routing service.
+ *
+ * @return all the hosts
+ */
+ Set<SocketAddress> getHosts();
+
+ /**
+ * Get the host to route the request by <i>key</i>.
+ *
+ * @param key
+ * key to route the request.
+ * @param rContext
+ * routing context.
+ * @return host to route the request
+ * @throws NoBrokersAvailableException
+ */
+ SocketAddress getHost(String key, RoutingContext rContext)
+ throws NoBrokersAvailableException;
+
+ /**
+ * Remove the host <i>address</i> for a specific <i>reason</i>.
+ *
+ * @param address
+ * host address to remove
+ * @param reason
+ * reason to remove the host
+ */
+ void removeHost(SocketAddress address, Throwable reason);
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java
new file mode 100644
index 0000000..4ac22ce
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java
@@ -0,0 +1,39 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import com.twitter.finagle.stats.StatsReceiver;
+
+class RoutingServiceProvider implements RoutingService.Builder {
+
+ final RoutingService routingService;
+
+ RoutingServiceProvider(RoutingService routingService) {
+ this.routingService = routingService;
+ }
+
+ @Override
+ public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
+ return this;
+ }
+
+ @Override
+ public RoutingService build() {
+ return routingService;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java
new file mode 100644
index 0000000..8e8edd3
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java
@@ -0,0 +1,88 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import com.twitter.common.zookeeper.ServerSet;
+import java.net.SocketAddress;
+
+/**
+ * Utils for routing services.
+ */
+public class RoutingUtils {
+
+ private static final int NUM_CONSISTENT_HASH_REPLICAS = 997;
+
+ /**
+ * Building routing service from <code>finagleNameStr</code>.
+ *
+ * @param finagleNameStr
+ * finagle name str of a service
+ * @return routing service builder
+ */
+ public static RoutingService.Builder buildRoutingService(String finagleNameStr) {
+ if (!finagleNameStr.startsWith("serverset!")
+ && !finagleNameStr.startsWith("inet!")
+ && !finagleNameStr.startsWith("zk!")) {
+ // We only support serverset based names at the moment
+ throw new UnsupportedOperationException("Finagle Name format not supported for name: " + finagleNameStr);
+ }
+ return buildRoutingService(new NameServerSet(finagleNameStr), true);
+ }
+
+ /**
+ * Building routing service from <code>serverSet</code>.
+ *
+ * @param serverSet
+ * server set of a service
+ * @return routing service builder
+ */
+ public static RoutingService.Builder buildRoutingService(ServerSet serverSet) {
+ return buildRoutingService(serverSet, false);
+ }
+
+ /**
+ * Building routing service from <code>address</code>.
+ *
+ * @param address
+ * host to route the requests
+ * @return routing service builder
+ */
+ public static RoutingService.Builder buildRoutingService(SocketAddress address) {
+ return SingleHostRoutingService.newBuilder().address(address);
+ }
+
+ /**
+ * Build routing service builder of a routing service <code>routingService</code>.
+ *
+ * @param routingService
+ * routing service to provide
+ * @return routing service builder
+ */
+ public static RoutingService.Builder buildRoutingService(RoutingService routingService) {
+ return new RoutingServiceProvider(routingService);
+ }
+
+ private static RoutingService.Builder buildRoutingService(ServerSet serverSet,
+ boolean resolveFromName) {
+ return ConsistentHashRoutingService.newBuilder()
+ .serverSet(serverSet)
+ .resolveFromName(resolveFromName)
+ .numReplicas(NUM_CONSISTENT_HASH_REPLICAS);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java
new file mode 100644
index 0000000..4fe8141
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java
@@ -0,0 +1,274 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+import org.apache.distributedlog.service.DLSocketAddress;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Routing Service based on a given {@link com.twitter.common.zookeeper.ServerSet}.
+ */
+class ServerSetRoutingService extends Thread implements RoutingService {
+
+ private static final Logger logger = LoggerFactory.getLogger(ServerSetRoutingService.class);
+
+ static ServerSetRoutingServiceBuilder newServerSetRoutingServiceBuilder() {
+ return new ServerSetRoutingServiceBuilder();
+ }
+
+ /**
+ * Builder to build {@link com.twitter.common.zookeeper.ServerSet} based routing service.
+ */
+ static class ServerSetRoutingServiceBuilder implements RoutingService.Builder {
+
+ private ServerSetWatcher serverSetWatcher;
+
+ private ServerSetRoutingServiceBuilder() {}
+
+ public ServerSetRoutingServiceBuilder serverSetWatcher(ServerSetWatcher serverSetWatcher) {
+ this.serverSetWatcher = serverSetWatcher;
+ return this;
+ }
+
+ @Override
+ public Builder statsReceiver(StatsReceiver statsReceiver) {
+ return this;
+ }
+
+ @Override
+ public RoutingService build() {
+ checkNotNull(serverSetWatcher, "No serverset watcher provided.");
+ return new ServerSetRoutingService(this.serverSetWatcher);
+ }
+ }
+
+ private static class HostComparator implements Comparator<SocketAddress> {
+
+ private static final HostComparator INSTANCE = new HostComparator();
+
+ @Override
+ public int compare(SocketAddress o1, SocketAddress o2) {
+ return o1.toString().compareTo(o2.toString());
+ }
+ }
+
+ private final ServerSetWatcher serverSetWatcher;
+
+ private final Set<SocketAddress> hostSet = new HashSet<SocketAddress>();
+ private List<SocketAddress> hostList = new ArrayList<SocketAddress>();
+ private final HashFunction hasher = Hashing.md5();
+
+ // Server Set Changes
+ private final AtomicReference<ImmutableSet<DLSocketAddress>> serverSetChange =
+ new AtomicReference<ImmutableSet<DLSocketAddress>>(null);
+ private final CountDownLatch changeLatch = new CountDownLatch(1);
+
+ // Listeners
+ protected final CopyOnWriteArraySet<RoutingListener> listeners =
+ new CopyOnWriteArraySet<RoutingListener>();
+
+ ServerSetRoutingService(ServerSetWatcher serverSetWatcher) {
+ super("ServerSetRoutingService");
+ this.serverSetWatcher = serverSetWatcher;
+ }
+
+ @Override
+ public Set<SocketAddress> getHosts() {
+ synchronized (hostSet) {
+ return ImmutableSet.copyOf(hostSet);
+ }
+ }
+
+ @Override
+ public void startService() {
+ start();
+ try {
+ if (!changeLatch.await(1, TimeUnit.MINUTES)) {
+ logger.warn("No serverset change received in 1 minute.");
+ }
+ } catch (InterruptedException e) {
+ logger.warn("Interrupted waiting first serverset change : ", e);
+ }
+ logger.info("{} Routing Service Started.", getClass().getSimpleName());
+ }
+
+ @Override
+ public void stopService() {
+ Thread.currentThread().interrupt();
+ try {
+ join();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ logger.warn("Interrupted on waiting serverset routing service to finish : ", e);
+ }
+ logger.info("{} Routing Service Stopped.", getClass().getSimpleName());
+ }
+
+ @Override
+ public RoutingService registerListener(RoutingListener listener) {
+ listeners.add(listener);
+ return this;
+ }
+
+ @Override
+ public RoutingService unregisterListener(RoutingListener listener) {
+ listeners.remove(listener);
+ return this;
+ }
+
+ @Override
+ public SocketAddress getHost(String key, RoutingContext rContext)
+ throws NoBrokersAvailableException {
+ SocketAddress address = null;
+ synchronized (hostSet) {
+ if (0 != hostList.size()) {
+ int hashCode = hasher.hashUnencodedChars(key).asInt();
+ int hostId = signSafeMod(hashCode, hostList.size());
+ address = hostList.get(hostId);
+ if (rContext.isTriedHost(address)) {
+ ArrayList<SocketAddress> newList = new ArrayList<SocketAddress>(hostList);
+ newList.remove(hostId);
+ // pickup a new host by rehashing it.
+ hostId = signSafeMod(hashCode, newList.size());
+ address = newList.get(hostId);
+ int i = hostId;
+ while (rContext.isTriedHost(address)) {
+ i = (i + 1) % newList.size();
+ if (i == hostId) {
+ address = null;
+ break;
+ }
+ address = newList.get(i);
+ }
+ }
+ }
+ }
+ if (null == address) {
+ throw new NoBrokersAvailableException("No host is available.");
+ }
+ return address;
+ }
+
+ @Override
+ public void removeHost(SocketAddress host, Throwable reason) {
+ synchronized (hostSet) {
+ if (hostSet.remove(host)) {
+ logger.info("Node {} left due to : ", host, reason);
+ }
+ hostList = new ArrayList<SocketAddress>(hostSet);
+ Collections.sort(hostList, HostComparator.INSTANCE);
+ logger.info("Host list becomes : {}.", hostList);
+ }
+ }
+
+ @Override
+ public void run() {
+ try {
+ serverSetWatcher.watch(new ServerSetWatcher.ServerSetMonitor() {
+ @Override
+ public void onChange(ImmutableSet<DLSocketAddress> serviceInstances) {
+ ImmutableSet<DLSocketAddress> lastValue = serverSetChange.getAndSet(serviceInstances);
+ if (null == lastValue) {
+ ImmutableSet<DLSocketAddress> mostRecentValue;
+ do {
+ mostRecentValue = serverSetChange.get();
+ performServerSetChange(mostRecentValue);
+ changeLatch.countDown();
+ } while (!serverSetChange.compareAndSet(mostRecentValue, null));
+ }
+ }
+ });
+ } catch (Exception e) {
+ logger.error("Fail to monitor server set : ", e);
+ Runtime.getRuntime().exit(-1);
+ }
+ }
+
+ protected synchronized void performServerSetChange(ImmutableSet<DLSocketAddress> serverSet) {
+ Set<SocketAddress> newSet = new HashSet<SocketAddress>();
+ for (DLSocketAddress serviceInstance : serverSet) {
+ newSet.add(serviceInstance.getSocketAddress());
+ }
+
+ Set<SocketAddress> removed;
+ Set<SocketAddress> added;
+ synchronized (hostSet) {
+ removed = Sets.difference(hostSet, newSet).immutableCopy();
+ added = Sets.difference(newSet, hostSet).immutableCopy();
+ for (SocketAddress node: removed) {
+ if (hostSet.remove(node)) {
+ logger.info("Node {} left.", node);
+ }
+ }
+ for (SocketAddress node: added) {
+ if (hostSet.add(node)) {
+ logger.info("Node {} joined.", node);
+ }
+ }
+ }
+
+ for (SocketAddress addr : removed) {
+ for (RoutingListener listener : listeners) {
+ listener.onServerLeft(addr);
+ }
+ }
+
+ for (SocketAddress addr : added) {
+ for (RoutingListener listener : listeners) {
+ listener.onServerJoin(addr);
+ }
+ }
+
+ synchronized (hostSet) {
+ hostList = new ArrayList<SocketAddress>(hostSet);
+ Collections.sort(hostList, HostComparator.INSTANCE);
+ logger.info("Host list becomes : {}.", hostList);
+ }
+
+ }
+
+ static int signSafeMod(long dividend, int divisor) {
+ int mod = (int) (dividend % divisor);
+
+ if (mod < 0) {
+ mod += divisor;
+ }
+
+ return mod;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java
new file mode 100644
index 0000000..77b7beb
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java
@@ -0,0 +1,71 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.distributedlog.service.DLSocketAddress;
+
+/**
+ * Watch on server set changes.
+ */
+public interface ServerSetWatcher {
+
+ /**
+ * Exception thrown when failed to monitor serverset.
+ */
+ class MonitorException extends Exception {
+
+ private static final long serialVersionUID = 392751505154339548L;
+
+ public MonitorException(String msg) {
+ super(msg);
+ }
+
+ public MonitorException(String msg, Throwable cause) {
+ super(msg, cause);
+ }
+ }
+
+ /**
+ * An interface to an object that is interested in receiving notification whenever the host set changes.
+ */
+ interface ServerSetMonitor {
+
+ /**
+ * Called when either the available set of services changes.
+ *
+ * <p>It happens either when a service dies or a new INSTANCE comes on-line or
+ * when an existing service advertises a status or health change.
+ *
+ * @param hostSet the current set of available ServiceInstances
+ */
+ void onChange(ImmutableSet<DLSocketAddress> hostSet);
+ }
+
+ /**
+ * Registers a monitor to receive change notices for this server set as long as this jvm process is alive.
+ *
+ * <p>Blocks until the initial server set can be gathered and delivered to the monitor.
+ * The monitor will be notified if the membership set or parameters of existing members have
+ * changed.
+ *
+ * @param monitor the server set monitor to call back when the host set changes
+ * @throws MonitorException if there is a problem monitoring the host set
+ */
+ void watch(final ServerSetMonitor monitor) throws MonitorException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java
new file mode 100644
index 0000000..753a1af
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java
@@ -0,0 +1,128 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.Sets;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+
+/**
+ * Single Host Routing Service.
+ */
+public class SingleHostRoutingService implements RoutingService {
+
+ public static SingleHostRoutingService of(SocketAddress address) {
+ return new SingleHostRoutingService(address);
+ }
+
+ /**
+ * Builder to build single host based routing service.
+ *
+ * @return builder to build single host based routing service.
+ */
+ public static Builder newBuilder() {
+ return new Builder();
+ }
+
+ /**
+ * Builder to build single host based routing service.
+ */
+ public static class Builder implements RoutingService.Builder {
+
+ private SocketAddress address;
+
+ private Builder() {}
+
+ public Builder address(SocketAddress address) {
+ this.address = address;
+ return this;
+ }
+
+ @Override
+ public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
+ return this;
+ }
+
+ @Override
+ public RoutingService build() {
+ checkNotNull(address, "Host is null");
+ return new SingleHostRoutingService(address);
+ }
+ }
+
+ private SocketAddress address;
+ private final CopyOnWriteArraySet<RoutingListener> listeners =
+ new CopyOnWriteArraySet<RoutingListener>();
+
+ SingleHostRoutingService(SocketAddress address) {
+ this.address = address;
+ }
+
+ public void setAddress(SocketAddress address) {
+ this.address = address;
+ }
+
+ @Override
+ public Set<SocketAddress> getHosts() {
+ return Sets.newHashSet(address);
+ }
+
+ @Override
+ public void startService() {
+ // no-op
+ for (RoutingListener listener : listeners) {
+ listener.onServerJoin(address);
+ }
+ }
+
+ @Override
+ public void stopService() {
+ // no-op
+ }
+
+ @Override
+ public RoutingService registerListener(RoutingListener listener) {
+ listeners.add(listener);
+ return this;
+ }
+
+ @Override
+ public RoutingService unregisterListener(RoutingListener listener) {
+ listeners.remove(listener);
+ return null;
+ }
+
+ @Override
+ public SocketAddress getHost(String key, RoutingContext rContext)
+ throws NoBrokersAvailableException {
+ if (rContext.isTriedHost(address)) {
+ throw new NoBrokersAvailableException("No hosts is available : routing context = " + rContext);
+ }
+ return address;
+ }
+
+ @Override
+ public void removeHost(SocketAddress address, Throwable reason) {
+ // no-op
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java
new file mode 100644
index 0000000..2fc8de0
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java
@@ -0,0 +1,49 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import com.twitter.finagle.Addr;
+import com.twitter.finagle.Address;
+import com.twitter.finagle.Addrs;
+import com.twitter.finagle.Name;
+import java.util.List;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A {@link Name} implementation for testing purpose.
+ */
+public class TestName implements Name {
+
+ private static final Logger LOG = LoggerFactory.getLogger(TestName.class);
+
+ private AbstractFunction1<Addr, BoxedUnit> callback = null;
+
+ public void changes(AbstractFunction1<Addr, BoxedUnit> callback) {
+ this.callback = callback;
+ }
+
+ public void changeAddrs(List<Address> addresses) {
+ if (null != callback) {
+ LOG.info("Sending a callback {}", addresses);
+ callback.apply(Addrs.newBoundAddr(addresses));
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java
new file mode 100644
index 0000000..1ff7c93
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java
@@ -0,0 +1,83 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import com.twitter.common.net.pool.DynamicHostSet;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.service.DLSocketAddress;
+import com.twitter.thrift.Endpoint;
+import com.twitter.thrift.ServiceInstance;
+import java.net.InetSocketAddress;
+import java.util.Set;
+
+/**
+ * Twitter {@link ServerSet} based watcher.
+ */
+public class TwitterServerSetWatcher implements ServerSetWatcher {
+
+ private final ServerSet serverSet;
+ private final boolean resolvedFromName;
+
+ /**
+ * Construct a {@link ServerSet} based watcher.
+ *
+ * @param serverSet server set.
+ * @param resolvedFromName whether to resolve hosts from {@link com.twitter.finagle.Name}.
+ */
+ public TwitterServerSetWatcher(ServerSet serverSet,
+ boolean resolvedFromName) {
+ this.serverSet = serverSet;
+ this.resolvedFromName = resolvedFromName;
+ }
+
+ /**
+ * Registers a monitor to receive change notices for this server set as long as this jvm process is alive.
+ *
+ * <p>Blocks until the initial server set can be gathered and delivered to the monitor.
+ * The monitor will be notified if the membership set or parameters of existing members have
+ * changed.
+ *
+ * @param monitor the server set monitor to call back when the host set changes
+ * @throws MonitorException if there is a problem monitoring the host set
+ */
+ public void watch(final ServerSetMonitor monitor)
+ throws MonitorException {
+ try {
+ serverSet.watch(new DynamicHostSet.HostChangeMonitor<ServiceInstance>() {
+ @Override
+ public void onChange(ImmutableSet<ServiceInstance> serviceInstances) {
+ Set<DLSocketAddress> dlServers = Sets.newHashSet();
+ for (ServiceInstance serviceInstance : serviceInstances) {
+ Endpoint endpoint = serviceInstance.getAdditionalEndpoints().get("thrift");
+ InetSocketAddress inetAddr =
+ new InetSocketAddress(endpoint.getHost(), endpoint.getPort());
+ int shardId = resolvedFromName ? -1 : serviceInstance.getShard();
+ DLSocketAddress address = new DLSocketAddress(shardId, inetAddr);
+ dlServers.add(address);
+ }
+ monitor.onChange(ImmutableSet.copyOf(dlServers));
+ }
+ });
+ } catch (DynamicHostSet.MonitorException me) {
+ throw new MonitorException("Failed to monitor server set : ", me);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java
new file mode 100644
index 0000000..352d755
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Routing Mechanisms to route the traffic to the owner of streams.
+ */
+package org.apache.distributedlog.client.routing;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java
new file mode 100644
index 0000000..93cdf7a
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.distributedlog.client.serverset;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.net.HostAndPort;
+import com.twitter.common.quantity.Amount;
+import com.twitter.common.quantity.Time;
+import com.twitter.common.zookeeper.ServerSet;
+import com.twitter.common.zookeeper.ServerSets;
+import com.twitter.common.zookeeper.ZooKeeperClient;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import org.apache.commons.lang.StringUtils;
+import org.apache.zookeeper.ZooDefs;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A wrapper over zookeeper client and its server set.
+ */
+public class DLZkServerSet {
+
+ private static final Logger logger = LoggerFactory.getLogger(DLZkServerSet.class);
+
+ static final String ZNODE_WRITE_PROXY = ".write_proxy";
+
+ private static String getZKServersFromDLUri(URI uri) {
+ return uri.getAuthority().replace(";", ",");
+ }
+
+ private static Iterable<InetSocketAddress> getZkAddresses(URI uri) {
+ String zkServers = getZKServersFromDLUri(uri);
+ String[] zkServerList = StringUtils.split(zkServers, ',');
+ ImmutableList.Builder<InetSocketAddress> builder = ImmutableList.builder();
+ for (String zkServer : zkServerList) {
+ HostAndPort hostAndPort = HostAndPort.fromString(zkServer).withDefaultPort(2181);
+ builder.add(InetSocketAddress.createUnresolved(
+ hostAndPort.getHostText(),
+ hostAndPort.getPort()));
+ }
+ return builder.build();
+ }
+
+ public static DLZkServerSet of(URI uri,
+ int zkSessionTimeoutMs) {
+ // Create zookeeper and server set
+ String zkPath = uri.getPath() + "/" + ZNODE_WRITE_PROXY;
+ Iterable<InetSocketAddress> zkAddresses = getZkAddresses(uri);
+ ZooKeeperClient zkClient =
+ new ZooKeeperClient(Amount.of(zkSessionTimeoutMs, Time.MILLISECONDS), zkAddresses);
+ ServerSet serverSet = ServerSets.create(zkClient, ZooDefs.Ids.OPEN_ACL_UNSAFE, zkPath);
+ return new DLZkServerSet(zkClient, serverSet);
+ }
+
+ private final ZooKeeperClient zkClient;
+ private final ServerSet zkServerSet;
+
+ public DLZkServerSet(ZooKeeperClient zkClient,
+ ServerSet zkServerSet) {
+ this.zkClient = zkClient;
+ this.zkServerSet = zkServerSet;
+ }
+
+ public ZooKeeperClient getZkClient() {
+ return zkClient;
+ }
+
+ public ServerSet getServerSet() {
+ return zkServerSet;
+ }
+
+ public void close() {
+ zkClient.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java
new file mode 100644
index 0000000..38a7544
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Utils related to server set.
+ */
+package org.apache.distributedlog.client.serverset;
[25/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/package-info.java
deleted file mode 100644
index 697d2e9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/package-info.java
+++ /dev/null
@@ -1,24 +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.
- */
-/**
- * BookKeeper related util functions.
- * <p>
- * <h2>Ledger Allocator</h2>
- *
- */
-package com.twitter.distributedlog.bk;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentListener.java
deleted file mode 100644
index 2196245..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentListener.java
+++ /dev/null
@@ -1,42 +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.callback;
-
-import com.twitter.distributedlog.LogSegmentMetadata;
-
-import java.util.List;
-
-/**
- * Listener on log segments changes for a given stream used by {@link com.twitter.distributedlog.BKLogReadHandler}
- */
-public interface LogSegmentListener {
-
- /**
- * Notified when <i>segments</i> updated. The new sorted log segments
- * list is returned in this method.
- *
- * @param segments
- * updated list of segments.
- */
- void onSegmentsUpdated(List<LogSegmentMetadata> segments);
-
- /**
- * Notified when the log stream is deleted.
- */
- void onLogStreamDeleted();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentNamesListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentNamesListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentNamesListener.java
deleted file mode 100644
index e38f305..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentNamesListener.java
+++ /dev/null
@@ -1,42 +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.callback;
-
-import org.apache.bookkeeper.versioning.Versioned;
-
-import java.util.List;
-
-/**
- * Listener on list of log segments changes for a given stream used by
- * {@link com.twitter.distributedlog.logsegment.LogSegmentMetadataStore}.
- */
-public interface LogSegmentNamesListener {
- /**
- * Notified when <i>segments</i> updated. The new log segments
- * list is returned in this method.
- *
- * @param segments
- * updated list of segments.
- */
- void onSegmentsUpdated(Versioned<List<String>> segments);
-
- /**
- * Notified when the log stream is deleted.
- */
- void onLogStreamDeleted();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/NamespaceListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/NamespaceListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/NamespaceListener.java
deleted file mode 100644
index fc63ff5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/NamespaceListener.java
+++ /dev/null
@@ -1,34 +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.callback;
-
-import com.google.common.annotations.Beta;
-
-import java.util.Iterator;
-
-@Beta
-public interface NamespaceListener {
-
- /**
- * Updated with latest streams.
- *
- * @param streams
- * latest list of streams under a given namespace.
- */
- void onStreamsChanged(Iterator<String> streams);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/ReadAheadCallback.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/ReadAheadCallback.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/ReadAheadCallback.java
deleted file mode 100644
index 7c46a1a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/ReadAheadCallback.java
+++ /dev/null
@@ -1,25 +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.callback;
-
-/**
- * ReadAhead Callback
- */
-public interface ReadAheadCallback {
- void resumeReadAhead();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/package-info.java
deleted file mode 100644
index 2724d43..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Callbacks for distributedlog operations.
- */
-package com.twitter.distributedlog.callback;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentBaseConfiguration.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentBaseConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentBaseConfiguration.java
deleted file mode 100644
index 91603c1..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentBaseConfiguration.java
+++ /dev/null
@@ -1,76 +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.config;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.commons.configuration.AbstractConfiguration;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Iterator;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * Configuration view built on concurrent hash map for fast thread-safe access.
- * Notes:
- * 1. Multi-property list aggregation will not work in this class. I.e. commons config
- * normally combines all properties with the same key into one list property automatically.
- * This class simply overwrites any existing mapping.
- */
-public class ConcurrentBaseConfiguration extends AbstractConfiguration {
- static final Logger LOG = LoggerFactory.getLogger(ConcurrentBaseConfiguration.class);
-
- private final ConcurrentHashMap<String, Object> map;
-
- public ConcurrentBaseConfiguration() {
- this.map = new ConcurrentHashMap<String, Object>();
- }
-
- @Override
- protected void addPropertyDirect(String key, Object value) {
- Preconditions.checkNotNull(value);
- map.put(key, value);
- }
-
- @Override
- public Object getProperty(String key) {
- return map.get(key);
- }
-
- @Override
- public Iterator getKeys() {
- return map.keySet().iterator();
- }
-
- @Override
- public boolean containsKey(String key) {
- return map.containsKey(key);
- }
-
- @Override
- public boolean isEmpty() {
- return map.isEmpty();
- }
-
- @Override
- protected void clearPropertyDirect(String key) {
- map.remove(key);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentConstConfiguration.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentConstConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentConstConfiguration.java
deleted file mode 100644
index a044a13..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentConstConfiguration.java
+++ /dev/null
@@ -1,31 +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.config;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.configuration.Configuration;
-
-/**
- * Invariant thread-safe view of some configuration.
- */
-public class ConcurrentConstConfiguration extends ConcurrentBaseConfiguration {
- public ConcurrentConstConfiguration(Configuration conf) {
- Preconditions.checkNotNull(conf);
- copy(conf);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationListener.java
deleted file mode 100644
index d4c44b7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationListener.java
+++ /dev/null
@@ -1,32 +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.config;
-
-/**
- * Configuration listener triggered when reloading configuration settings.
- */
-public interface ConfigurationListener {
-
- /**
- * Reload the configuration.
- *
- * @param conf configuration to reload
- */
- void onReload(ConcurrentBaseConfiguration conf);
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationSubscription.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationSubscription.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationSubscription.java
deleted file mode 100644
index dadfe81..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationSubscription.java
+++ /dev/null
@@ -1,186 +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.config;
-
-import java.io.FileNotFoundException;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.Iterator;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.FileConfiguration;
-import org.apache.commons.configuration.reloading.FileChangedReloadingStrategy;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * ConfigurationSubscription publishes a reloading, thread-safe view of file configuration. The class
- * periodically calls FileConfiguration.reload on the underlying conf, and propagates changes to the
- * concurrent config. The configured FileChangedReloadingStrategy ensures that file config will only
- * be reloaded if something changed.
- * Notes:
- * 1. Reload schedule is never terminated. The assumption is a finite number of these are started
- * at the calling layer, and terminated only once the executor service is shut down.
- * 2. The underlying FileConfiguration is not at all thread-safe, so its important to ensure access
- * to this object is always single threaded.
- */
-public class ConfigurationSubscription {
- static final Logger LOG = LoggerFactory.getLogger(ConfigurationSubscription.class);
-
- private final ConcurrentBaseConfiguration viewConfig;
- private final ScheduledExecutorService executorService;
- private final int reloadPeriod;
- private final TimeUnit reloadUnit;
- private final List<FileConfigurationBuilder> fileConfigBuilders;
- private final List<FileConfiguration> fileConfigs;
- private final CopyOnWriteArraySet<ConfigurationListener> confListeners;
-
- public ConfigurationSubscription(ConcurrentBaseConfiguration viewConfig,
- List<FileConfigurationBuilder> fileConfigBuilders,
- ScheduledExecutorService executorService,
- int reloadPeriod,
- TimeUnit reloadUnit)
- throws ConfigurationException {
- Preconditions.checkNotNull(fileConfigBuilders);
- Preconditions.checkArgument(!fileConfigBuilders.isEmpty());
- Preconditions.checkNotNull(executorService);
- Preconditions.checkNotNull(viewConfig);
- this.viewConfig = viewConfig;
- this.executorService = executorService;
- this.reloadPeriod = reloadPeriod;
- this.reloadUnit = reloadUnit;
- this.fileConfigBuilders = fileConfigBuilders;
- this.fileConfigs = Lists.newArrayListWithExpectedSize(this.fileConfigBuilders.size());
- this.confListeners = new CopyOnWriteArraySet<ConfigurationListener>();
- reload();
- scheduleReload();
- }
-
- public void registerListener(ConfigurationListener listener) {
- this.confListeners.add(listener);
- }
-
- public void unregisterListener(ConfigurationListener listener) {
- this.confListeners.remove(listener);
- }
-
- private boolean initConfig() {
- if (fileConfigs.isEmpty()) {
- try {
- for (FileConfigurationBuilder fileConfigBuilder : fileConfigBuilders) {
- FileConfiguration fileConfig = fileConfigBuilder.getConfiguration();
- FileChangedReloadingStrategy reloadingStrategy = new FileChangedReloadingStrategy();
- reloadingStrategy.setRefreshDelay(0);
- fileConfig.setReloadingStrategy(reloadingStrategy);
- fileConfigs.add(fileConfig);
- }
- } catch (ConfigurationException ex) {
- if (!fileNotFound(ex)) {
- LOG.error("Config init failed {}", ex);
- }
- }
- }
- return !fileConfigs.isEmpty();
- }
-
- private void scheduleReload() {
- executorService.scheduleAtFixedRate(new Runnable() {
- @Override
- public void run() {
- reload();
- }
- }, 0, reloadPeriod, reloadUnit);
- }
-
- @VisibleForTesting
- void reload() {
- // No-op if already loaded.
- if (!initConfig()) {
- return;
- }
- // Reload if config exists.
- Set<String> confKeys = Sets.newHashSet();
- for (FileConfiguration fileConfig : fileConfigs) {
- LOG.debug("Check and reload config, file={}, lastModified={}", fileConfig.getFile(),
- fileConfig.getFile().lastModified());
- fileConfig.reload();
- // load keys
- Iterator keyIter = fileConfig.getKeys();
- while (keyIter.hasNext()) {
- String key = (String) keyIter.next();
- confKeys.add(key);
- }
- }
- // clear unexisted keys
- Iterator viewIter = viewConfig.getKeys();
- while (viewIter.hasNext()) {
- String key = (String) viewIter.next();
- if (!confKeys.contains(key)) {
- clearViewProperty(key);
- }
- }
- LOG.info("Reload features : {}", confKeys);
- // load keys from files
- for (FileConfiguration fileConfig : fileConfigs) {
- try {
- loadView(fileConfig);
- } catch (Exception ex) {
- if (!fileNotFound(ex)) {
- LOG.error("Config reload failed for file {}", fileConfig.getFileName(), ex);
- }
- }
- }
- for (ConfigurationListener listener : confListeners) {
- listener.onReload(viewConfig);
- }
- }
-
- private boolean fileNotFound(Exception ex) {
- return ex instanceof FileNotFoundException ||
- ex.getCause() != null && ex.getCause() instanceof FileNotFoundException;
- }
-
- private void loadView(FileConfiguration fileConfig) {
- Iterator fileIter = fileConfig.getKeys();
- while (fileIter.hasNext()) {
- String key = (String) fileIter.next();
- setViewProperty(fileConfig, key, fileConfig.getProperty(key));
- }
- }
-
- private void clearViewProperty(String key) {
- LOG.debug("Removing property, key={}", key);
- viewConfig.clearProperty(key);
- }
-
- private void setViewProperty(FileConfiguration fileConfig,
- String key,
- Object value) {
- if (!viewConfig.containsKey(key) || !viewConfig.getProperty(key).equals(value)) {
- LOG.debug("Setting property, key={} value={}", key, fileConfig.getProperty(key));
- viewConfig.setProperty(key, fileConfig.getProperty(key));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicConfigurationFactory.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicConfigurationFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicConfigurationFactory.java
deleted file mode 100644
index 2510f74..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicConfigurationFactory.java
+++ /dev/null
@@ -1,91 +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.config;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.net.MalformedURLException;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.configuration.ConfigurationException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Encapsulates creation of DynamicDistributedLogConfiguration instances. Ensures one instance per
- * factory.
- * Notes:
- * Once loaded, stays loaded until shutdown. Caller ensures small finite number of configs are created.
- */
-public class DynamicConfigurationFactory {
- private static final Logger LOG = LoggerFactory.getLogger(DynamicConfigurationFactory.class);
-
- private final Map<String, DynamicDistributedLogConfiguration> dynamicConfigs;
- private final List<ConfigurationSubscription> subscriptions;
- private final ScheduledExecutorService executorService;
- private final int reloadPeriod;
- private final TimeUnit reloadUnit;
-
- public DynamicConfigurationFactory(ScheduledExecutorService executorService, int reloadPeriod, TimeUnit reloadUnit) {
- this.executorService = executorService;
- this.reloadPeriod = reloadPeriod;
- this.reloadUnit = reloadUnit;
- this.dynamicConfigs = new HashMap<String, DynamicDistributedLogConfiguration>();
- this.subscriptions = new LinkedList<ConfigurationSubscription>();
- }
-
- public synchronized Optional<DynamicDistributedLogConfiguration> getDynamicConfiguration(
- String configPath,
- ConcurrentBaseConfiguration defaultConf) throws ConfigurationException {
- Preconditions.checkNotNull(configPath);
- try {
- if (!dynamicConfigs.containsKey(configPath)) {
- File configFile = new File(configPath);
- FileConfigurationBuilder properties =
- new PropertiesConfigurationBuilder(configFile.toURI().toURL());
- DynamicDistributedLogConfiguration dynConf =
- new DynamicDistributedLogConfiguration(defaultConf);
- List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(properties);
- ConfigurationSubscription subscription = new ConfigurationSubscription(
- dynConf, fileConfigBuilders, executorService, reloadPeriod, reloadUnit);
- subscriptions.add(subscription);
- dynamicConfigs.put(configPath, dynConf);
- LOG.info("Loaded dynamic configuration at {}", configPath);
- }
- return Optional.of(dynamicConfigs.get(configPath));
- } catch (MalformedURLException ex) {
- throw new ConfigurationException(ex);
- }
- }
-
- public synchronized Optional<DynamicDistributedLogConfiguration> getDynamicConfiguration(String configPath) throws ConfigurationException {
- return getDynamicConfiguration(configPath, new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicDistributedLogConfiguration.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicDistributedLogConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicDistributedLogConfiguration.java
deleted file mode 100644
index ca43cfa..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicDistributedLogConfiguration.java
+++ /dev/null
@@ -1,356 +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.config;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.bk.QuorumConfig;
-
-import static com.twitter.distributedlog.DistributedLogConfiguration.*;
-
-/**
- * Whitelist dynamic configuration by adding an accessor to this class.
- */
-public class DynamicDistributedLogConfiguration extends ConcurrentBaseConfiguration {
-
- private final ConcurrentBaseConfiguration defaultConfig;
-
- public DynamicDistributedLogConfiguration(ConcurrentBaseConfiguration defaultConfig) {
- this.defaultConfig = defaultConfig;
- }
-
- private static int getInt(ConcurrentBaseConfiguration configuration,
- String newKey,
- String oldKey,
- int defaultValue) {
- return configuration.getInt(newKey, configuration.getInt(oldKey, defaultValue));
- }
-
- /**
- * Get retention period in hours
- *
- * @return retention period in hours
- */
- public int getRetentionPeriodHours() {
- return getInt(
- this,
- BKDL_RETENTION_PERIOD_IN_HOURS,
- BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
- getInt(defaultConfig,
- BKDL_RETENTION_PERIOD_IN_HOURS,
- BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
- BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT)
- );
- }
-
- /**
- * A lower threshold bytes per second limit on writes to the distributedlog proxy.
- *
- * @return Bytes per second write limit
- */
- public int getBpsSoftWriteLimit() {
- return getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_WRITE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_WRITE_LIMIT,
- DistributedLogConfiguration.BKDL_BPS_SOFT_WRITE_LIMIT_DEFAULT));
- }
-
- /**
- * An upper threshold bytes per second limit on writes to the distributedlog proxy.
- *
- * @return Bytes per second write limit
- */
- public int getBpsHardWriteLimit() {
- return getInt(DistributedLogConfiguration.BKDL_BPS_HARD_WRITE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_HARD_WRITE_LIMIT,
- DistributedLogConfiguration.BKDL_BPS_HARD_WRITE_LIMIT_DEFAULT));
- }
-
- /**
- * A lower threshold requests per second limit on writes to the distributedlog proxy.
- *
- * @return Requests per second write limit
- */
- public int getRpsSoftWriteLimit() {
- return getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_WRITE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_WRITE_LIMIT,
- DistributedLogConfiguration.BKDL_RPS_SOFT_WRITE_LIMIT_DEFAULT));
- }
-
- /**
- * An upper threshold requests per second limit on writes to the distributedlog proxy.
- *
- * @return Requests per second write limit
- */
- public int getRpsHardWriteLimit() {
- return getInt(DistributedLogConfiguration.BKDL_RPS_HARD_WRITE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_HARD_WRITE_LIMIT,
- DistributedLogConfiguration.BKDL_RPS_HARD_WRITE_LIMIT_DEFAULT));
- }
-
- /**
- * A lower threshold requests per second limit on writes to the distributedlog proxy globally.
- *
- * @return Requests per second write limit
- */
- public int getRpsSoftServiceLimit() {
- return getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_SERVICE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_SERVICE_LIMIT,
- DistributedLogConfiguration.BKDL_RPS_SOFT_SERVICE_LIMIT_DEFAULT));
- }
-
- /**
- * An upper threshold requests per second limit on writes to the distributedlog proxy globally.
- *
- * @return Requests per second write limit
- */
- public int getRpsHardServiceLimit() {
- return getInt(DistributedLogConfiguration.BKDL_RPS_HARD_SERVICE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_HARD_SERVICE_LIMIT,
- DistributedLogConfiguration.BKDL_RPS_HARD_SERVICE_LIMIT_DEFAULT));
- }
-
- /**
- * When 60min average rps for the entire service instance hits this value, new streams will be
- * rejected.
- *
- * @return Requests per second limit
- */
- public int getRpsStreamAcquireServiceLimit() {
- return getInt(DistributedLogConfiguration.BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT,
- DistributedLogConfiguration.BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT));
- }
-
- /**
- * A lower threshold bytes per second limit on writes to the distributedlog proxy globally.
- *
- * @return Bytes per second write limit
- */
- public int getBpsSoftServiceLimit() {
- return getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_SERVICE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_SERVICE_LIMIT,
- DistributedLogConfiguration.BKDL_BPS_SOFT_SERVICE_LIMIT_DEFAULT));
- }
-
- /**
- * An upper threshold bytes per second limit on writes to the distributedlog proxy globally.
- *
- * @return Bytes per second write limit
- */
- public int getBpsHardServiceLimit() {
- return getInt(DistributedLogConfiguration.BKDL_BPS_HARD_SERVICE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_HARD_SERVICE_LIMIT,
- DistributedLogConfiguration.BKDL_BPS_HARD_SERVICE_LIMIT_DEFAULT));
- }
-
- /**
- * When 60min average bps for the entire service instance hits this value, new streams will be
- * rejected.
- *
- * @return Bytes per second limit
- */
- public int getBpsStreamAcquireServiceLimit() {
- return getInt(DistributedLogConfiguration.BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT,
- DistributedLogConfiguration.BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT));
- }
-
- /**
- * Get percent of write bytes which should be delayed by BKDL_EI_INJECTED_WRITE_DELAY_MS.
- *
- * @return percent of writes to delay.
- */
- public double getEIInjectedWriteDelayPercent() {
- return getDouble(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_PERCENT,
- defaultConfig.getDouble(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_PERCENT,
- DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_PERCENT_DEFAULT));
- }
-
- /**
- * Get amount of time to delay writes for in writer failure injection.
- *
- * @return millis to delay writes for.
- */
- public int getEIInjectedWriteDelayMs() {
- return getInt(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_MS,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_MS,
- DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_MS_DEFAULT));
- }
-
- /**
- * Get output buffer size
- *
- * @return buffer size
- */
- public int getOutputBufferSize() {
- return getInt(
- this,
- BKDL_OUTPUT_BUFFER_SIZE,
- BKDL_OUTPUT_BUFFER_SIZE_OLD,
- getInt(defaultConfig,
- BKDL_OUTPUT_BUFFER_SIZE,
- BKDL_OUTPUT_BUFFER_SIZE_OLD,
- BKDL_OUTPUT_BUFFER_SIZE_DEFAULT)
- );
- }
-
- /**
- * Get Periodic Log Flush Frequency in seconds
- *
- * @return periodic flush frequency
- */
- public int getPeriodicFlushFrequencyMilliSeconds() {
- return getInt(DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
- defaultConfig.getInt(DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
- DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT));
- }
-
- /**
- * Get the number of entries that readahead worker reads as a batch from bookkeeper
- *
- * @return the batch size
- */
- public int getReadAheadBatchSize() {
- return getInt(
- this,
- BKDL_READAHEAD_BATCHSIZE,
- BKDL_READAHEAD_BATCHSIZE_OLD,
- getInt(defaultConfig,
- BKDL_READAHEAD_BATCHSIZE,
- BKDL_READAHEAD_BATCHSIZE_OLD,
- BKDL_READAHEAD_BATCHSIZE_DEFAULT)
- );
- }
-
- /**
- * Get the maximum number of {@link com.twitter.distributedlog.LogRecord } that readahead worker will cache.
- *
- * @return the maximum number
- */
- public int getReadAheadMaxRecords() {
- return getInt(
- this,
- BKDL_READAHEAD_MAX_RECORDS,
- BKDL_READAHEAD_MAX_RECORDS_OLD,
- getInt(defaultConfig,
- BKDL_READAHEAD_MAX_RECORDS,
- BKDL_READAHEAD_MAX_RECORDS_OLD,
- BKDL_READAHEAD_MAX_RECORDS_DEFAULT)
- );
- }
-
- /**
- * Whether to enable ledger allocator pool or not.
- * It is disabled by default.
- *
- * @return whether using ledger allocator pool or not.
- */
- public boolean getEnableLedgerAllocatorPool() {
- return getBoolean(BKDL_ENABLE_LEDGER_ALLOCATOR_POOL,
- defaultConfig.getBoolean(
- BKDL_ENABLE_LEDGER_ALLOCATOR_POOL,
- BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT));
- }
-
- /**
- * Get the quorum config.
- *
- * @return quorum config
- */
- public QuorumConfig getQuorumConfig() {
- int ensembleSize = getInt(
- this,
- BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
- BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
- getInt(defaultConfig,
- BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
- BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
- BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT));
- int writeQuorumSize = getInt(
- this,
- BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
- BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
- getInt(defaultConfig,
- BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
- BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
- BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT));
- int ackQuorumSize = getInt(
- this,
- BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
- BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
- getInt(defaultConfig,
- BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
- BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
- BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT));
- return new QuorumConfig(ensembleSize, writeQuorumSize, ackQuorumSize);
- }
-
- /**
- * Get the maximum number of partitions of each stream allowed to be acquired per proxy.
- *
- * @return maximum number of partitions of each stream allowed to be acquired
- * @see DistributedLogConfiguration#getMaxAcquiredPartitionsPerProxy()
- */
- public int getMaxAcquiredPartitionsPerProxy() {
- return getInt(
- BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY,
- defaultConfig.getInt(
- BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY,
- BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT)
- );
- }
-
- /**
- * Get the maximum number of partitions of each stream allowed to cache per proxy.
- *
- * @return maximum number of partitions of each stream allowed to cache
- * @see DistributedLogConfiguration#getMaxAcquiredPartitionsPerProxy()
- */
- public int getMaxCachedPartitionsPerProxy() {
- return getInt(
- BKDL_MAX_CACHED_PARTITIONS_PER_PROXY,
- defaultConfig.getInt(
- BKDL_MAX_CACHED_PARTITIONS_PER_PROXY,
- BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT)
- );
- }
-
- /**
- * Check whether the durable write is enabled.
- *
- * @return true if durable write is enabled. otherwise, false.
- */
- public boolean isDurableWriteEnabled() {
- return getBoolean(BKDL_IS_DURABLE_WRITE_ENABLED,
- defaultConfig.getBoolean(
- BKDL_IS_DURABLE_WRITE_ENABLED,
- BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT));
- }
-
- /**
- * Get the flag whether to deserialize recordset on reads.
- *
- * @return flag whether to deserialize recordset on reads.
- */
- public boolean getDeserializeRecordSetOnReads() {
- return getBoolean(BKDL_DESERIALIZE_RECORDSET_ON_READS,
- defaultConfig.getBoolean(
- BKDL_DESERIALIZE_RECORDSET_ON_READS,
- BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/FileConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/FileConfigurationBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/FileConfigurationBuilder.java
deleted file mode 100644
index b3c4e6c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/FileConfigurationBuilder.java
+++ /dev/null
@@ -1,28 +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.config;
-
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.FileConfiguration;
-
-/**
- * Abstract out FileConfiguration subclass construction.
- */
-public interface FileConfigurationBuilder {
- FileConfiguration getConfiguration() throws ConfigurationException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/PropertiesConfigurationBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/PropertiesConfigurationBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/PropertiesConfigurationBuilder.java
deleted file mode 100644
index 6efaa20..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/PropertiesConfigurationBuilder.java
+++ /dev/null
@@ -1,40 +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.config;
-
-import java.net.URL;
-
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.FileConfiguration;
-import org.apache.commons.configuration.PropertiesConfiguration;
-
-/**
- * Hide PropertiesConfiguration dependency.
- */
-public class PropertiesConfigurationBuilder implements FileConfigurationBuilder {
- private URL url;
-
- public PropertiesConfigurationBuilder(URL url) {
- this.url = url;
- }
-
- @Override
- public FileConfiguration getConfiguration() throws ConfigurationException {
- return new PropertiesConfiguration(url);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/package-info.java
deleted file mode 100644
index b4f77b4..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Configuration
- */
-package com.twitter.distributedlog.config;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/exceptions/ZKException.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/exceptions/ZKException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/exceptions/ZKException.java
deleted file mode 100644
index 8ed1610..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/exceptions/ZKException.java
+++ /dev/null
@@ -1,54 +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.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.Code;
-
-/**
- * TODO: move ZKException to distributedlog-protocol
- */
-public class ZKException extends DLException {
-
- private static final long serialVersionUID = 7542748595054923600L;
-
- final KeeperException.Code code;
-
- public ZKException(String msg, Code code) {
- super(StatusCode.ZOOKEEPER_ERROR, msg + " : " + code);
- this.code = code;
- }
-
- public ZKException(String msg, KeeperException exception) {
- super(StatusCode.ZOOKEEPER_ERROR, msg, exception);
- this.code = exception.code();
- }
-
- public Code getKeeperExceptionCode() {
- return this.code;
- }
-
- public static boolean isRetryableZKException(ZKException zke) {
- KeeperException.Code code = zke.getKeeperExceptionCode();
- return KeeperException.Code.CONNECTIONLOSS == code ||
- KeeperException.Code.OPERATIONTIMEOUT == code ||
- KeeperException.Code.SESSIONEXPIRED == code ||
- KeeperException.Code.SESSIONMOVED == code;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/AbstractFeatureProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/AbstractFeatureProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/AbstractFeatureProvider.java
deleted file mode 100644
index f484307..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/AbstractFeatureProvider.java
+++ /dev/null
@@ -1,103 +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.feature;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import org.apache.bookkeeper.feature.CacheableFeatureProvider;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.commons.configuration.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-
-/**
- * Decider based feature provider
- */
-public abstract class AbstractFeatureProvider<T extends Feature> extends CacheableFeatureProvider<T> {
-
- protected static final Logger logger = LoggerFactory.getLogger(AbstractFeatureProvider.class);
-
- public static FeatureProvider getFeatureProvider(String rootScope,
- DistributedLogConfiguration conf,
- StatsLogger statsLogger)
- throws IOException {
- Class<? extends FeatureProvider> featureProviderClass;
- try {
- featureProviderClass = conf.getFeatureProviderClass();
- } catch (ConfigurationException e) {
- throw new IOException("Can't initialize the feature provider : ", e);
- }
- // create feature provider
- Constructor<? extends FeatureProvider> constructor;
- try {
- constructor = featureProviderClass.getDeclaredConstructor(
- String.class,
- DistributedLogConfiguration.class,
- StatsLogger.class);
- } catch (NoSuchMethodException e) {
- throw new IOException("No constructor found for feature provider class " + featureProviderClass + " : ", e);
- }
- try {
- return constructor.newInstance(rootScope, conf, statsLogger);
- } catch (InstantiationException e) {
- throw new IOException("Failed to instantiate feature provider : ", e);
- } catch (IllegalAccessException e) {
- throw new IOException("Encountered illegal access when instantiating feature provider : ", e);
- } catch (InvocationTargetException e) {
- Throwable targetException = e.getTargetException();
- if (targetException instanceof IOException) {
- throw (IOException) targetException;
- } else {
- throw new IOException("Encountered invocation target exception while instantiating feature provider : ", e);
- }
- }
- }
-
- protected final DistributedLogConfiguration conf;
- protected final StatsLogger statsLogger;
-
- protected AbstractFeatureProvider(String rootScope,
- DistributedLogConfiguration conf,
- StatsLogger statsLogger) {
- super(rootScope);
- this.conf = conf;
- this.statsLogger = statsLogger;
- }
-
- /**
- * Start the feature provider.
- *
- * @throws IOException when failed to start the feature provider.
- */
- public void start() throws IOException {
- // no-op
- }
-
- /**
- * Stop the feature provider.
- */
- public void stop() {
- // no-op
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/ConfigurationFeatureProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/ConfigurationFeatureProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/ConfigurationFeatureProvider.java
deleted file mode 100644
index 02a4d79..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/ConfigurationFeatureProvider.java
+++ /dev/null
@@ -1,76 +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.feature;
-
-import com.twitter.distributedlog.config.ConcurrentBaseConfiguration;
-import org.apache.bookkeeper.feature.CacheableFeatureProvider;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Feature Provider that load features from configuration
- */
-class ConfigurationFeatureProvider extends CacheableFeatureProvider {
-
- private static final Logger logger = LoggerFactory.getLogger(ConfigurationFeatureProvider.class);
-
- static SettableFeature makeFeature(ConcurrentBaseConfiguration featuresConf,
- ConcurrentMap<String, SettableFeature> features,
- String featureName) {
- SettableFeature feature = features.get(featureName);
- if (null == feature) {
- int availability = featuresConf.getInt(featureName, 0);
- feature = new SettableFeature(featureName, availability);
- SettableFeature oldFeature =
- features.putIfAbsent(featureName, feature);
- if (null != oldFeature) {
- feature = oldFeature;
- } else {
- logger.info("Load feature {}={}", featureName, availability);
- }
- }
- return feature;
- }
-
- private final ConcurrentBaseConfiguration featuresConf;
- private final ConcurrentMap<String, SettableFeature> features;
-
- ConfigurationFeatureProvider(String rootScope,
- ConcurrentBaseConfiguration featuresConf,
- ConcurrentMap<String, SettableFeature> features) {
- super(rootScope);
- this.featuresConf = featuresConf;
- this.features = features;
- }
-
- @Override
- protected Feature makeFeature(String featureName) {
- return makeFeature(featuresConf, features, featureName);
- }
-
- @Override
- protected FeatureProvider makeProvider(String fullScopeName) {
- return new ConfigurationFeatureProvider(
- fullScopeName, featuresConf, features);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/CoreFeatureKeys.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/CoreFeatureKeys.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/CoreFeatureKeys.java
deleted file mode 100644
index 49b3354..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/CoreFeatureKeys.java
+++ /dev/null
@@ -1,29 +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.feature;
-
-/**
- * List of feature keys used by distributedlog core
- */
-public enum CoreFeatureKeys {
- // @Deprecated: bkc features are managed by bookkeeper prefixed with a scope
- DISABLE_DURABILITY_ENFORCEMENT,
- // disabling logsegment rolling
- DISABLE_LOGSEGMENT_ROLLING,
- DISABLE_WRITE_LIMIT,
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DefaultFeatureProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DefaultFeatureProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DefaultFeatureProvider.java
deleted file mode 100644
index 6554eaa..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DefaultFeatureProvider.java
+++ /dev/null
@@ -1,47 +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.feature;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Default feature provider which disable all features by default.
- */
-public class DefaultFeatureProvider extends AbstractFeatureProvider {
-
- public DefaultFeatureProvider(String rootScope,
- DistributedLogConfiguration conf,
- StatsLogger statsLogger) {
- super(rootScope, conf, statsLogger);
- }
-
- @Override
- protected Feature makeFeature(String featureName) {
- return new SettableFeature(featureName, 0);
- }
-
- @Override
- protected FeatureProvider makeProvider(String fullScopeName) {
- return new SettableFeatureProvider(fullScopeName, 0);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DynamicConfigurationFeatureProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DynamicConfigurationFeatureProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DynamicConfigurationFeatureProvider.java
deleted file mode 100644
index 1eeb155..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DynamicConfigurationFeatureProvider.java
+++ /dev/null
@@ -1,132 +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.feature;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.config.ConcurrentBaseConfiguration;
-import com.twitter.distributedlog.config.ConfigurationListener;
-import com.twitter.distributedlog.config.ConfigurationSubscription;
-import com.twitter.distributedlog.config.FileConfigurationBuilder;
-import com.twitter.distributedlog.config.PropertiesConfigurationBuilder;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.commons.configuration.ConfigurationException;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Feature Provider based dynamic configuration.
- */
-public class DynamicConfigurationFeatureProvider extends AbstractFeatureProvider
- implements ConfigurationListener {
-
- private final ConcurrentBaseConfiguration featuresConf;
- private ConfigurationSubscription featuresConfSubscription;
- private final ConcurrentMap<String, SettableFeature> features;
- private final ScheduledExecutorService executorService;
-
- public DynamicConfigurationFeatureProvider(String rootScope,
- DistributedLogConfiguration conf,
- StatsLogger statsLogger) {
- super(rootScope, conf, statsLogger);
- this.features = new ConcurrentHashMap<String, SettableFeature>();
- this.featuresConf = new ConcurrentBaseConfiguration();
- this.executorService = Executors.newSingleThreadScheduledExecutor(
- new ThreadFactoryBuilder().setNameFormat("DynamicConfigurationFeatureProvider-%d").build());
- }
-
- ConcurrentBaseConfiguration getFeatureConf() {
- return featuresConf;
- }
-
- ConfigurationSubscription getFeatureConfSubscription() {
- return featuresConfSubscription;
- }
-
- @Override
- public void start() throws IOException {
- List<FileConfigurationBuilder> fileConfigBuilders =
- Lists.newArrayListWithExpectedSize(2);
- String baseConfigPath = conf.getFileFeatureProviderBaseConfigPath();
- Preconditions.checkNotNull(baseConfigPath);
- File baseConfigFile = new File(baseConfigPath);
- FileConfigurationBuilder baseProperties =
- new PropertiesConfigurationBuilder(baseConfigFile.toURI().toURL());
- fileConfigBuilders.add(baseProperties);
- String overlayConfigPath = conf.getFileFeatureProviderOverlayConfigPath();
- if (null != overlayConfigPath) {
- File overlayConfigFile = new File(overlayConfigPath);
- FileConfigurationBuilder overlayProperties =
- new PropertiesConfigurationBuilder(overlayConfigFile.toURI().toURL());
- fileConfigBuilders.add(overlayProperties);
- }
- try {
- this.featuresConfSubscription = new ConfigurationSubscription(
- this.featuresConf,
- fileConfigBuilders,
- executorService,
- conf.getDynamicConfigReloadIntervalSec(),
- TimeUnit.SECONDS);
- } catch (ConfigurationException e) {
- throw new IOException("Failed to register subscription on features configuration");
- }
- this.featuresConfSubscription.registerListener(this);
- }
-
- @Override
- public void stop() {
- this.executorService.shutdown();
- }
-
- @Override
- public void onReload(ConcurrentBaseConfiguration conf) {
- for (Map.Entry<String, SettableFeature> feature : features.entrySet()) {
- String featureName = feature.getKey();
- int availability = conf.getInt(featureName, 0);
- if (availability != feature.getValue().availability()) {
- feature.getValue().set(availability);
- logger.info("Reload feature {}={}", featureName, availability);
- }
- }
- }
-
- @Override
- protected Feature makeFeature(String featureName) {
- return ConfigurationFeatureProvider.makeFeature(
- featuresConf, features, featureName);
- }
-
- @Override
- protected FeatureProvider makeProvider(String fullScopeName) {
- return new ConfigurationFeatureProvider(
- fullScopeName, featuresConf, features);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/package-info.java
deleted file mode 100644
index e8d8134..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Feature & FeatureProvider for distributedlog
- */
-package com.twitter.distributedlog.feature;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/CloseAsyncCloseableFunction.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/CloseAsyncCloseableFunction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/CloseAsyncCloseableFunction.java
deleted file mode 100644
index 698a088..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/CloseAsyncCloseableFunction.java
+++ /dev/null
@@ -1,51 +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.function;
-
-import com.twitter.distributedlog.io.AsyncCloseable;
-import scala.Function0;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.BoxedUnit;
-
-/**
- * Function to close {@link com.twitter.distributedlog.io.AsyncCloseable}
- */
-public class CloseAsyncCloseableFunction extends AbstractFunction0<BoxedUnit> {
-
- /**
- * Return a function to close an {@link AsyncCloseable}.
- *
- * @param closeable closeable to close
- * @return function to close an {@link AsyncCloseable}
- */
- public static Function0<BoxedUnit> of(AsyncCloseable closeable) {
- return new CloseAsyncCloseableFunction(closeable);
- }
-
- private final AsyncCloseable closeable;
-
- private CloseAsyncCloseableFunction(AsyncCloseable closeable) {
- this.closeable = closeable;
- }
-
- @Override
- public BoxedUnit apply() {
- closeable.asyncClose();
- return BoxedUnit.UNIT;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/DefaultValueMapFunction.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/DefaultValueMapFunction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/DefaultValueMapFunction.java
deleted file mode 100644
index f08cd0f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/DefaultValueMapFunction.java
+++ /dev/null
@@ -1,41 +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.function;
-
-import scala.runtime.AbstractFunction1;
-
-/**
- * Map Function return default value
- */
-public class DefaultValueMapFunction<T, R> extends AbstractFunction1<T, R> {
-
- public static <T, R> DefaultValueMapFunction<T, R> of(R defaultValue) {
- return new DefaultValueMapFunction<T, R>(defaultValue);
- }
-
- private final R defaultValue;
-
- private DefaultValueMapFunction(R defaultValue) {
- this.defaultValue = defaultValue;
- }
-
- @Override
- public R apply(T any) {
- return defaultValue;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetLastTxIdFunction.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetLastTxIdFunction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetLastTxIdFunction.java
deleted file mode 100644
index bc77d6a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetLastTxIdFunction.java
+++ /dev/null
@@ -1,43 +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.function;
-
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import scala.runtime.AbstractFunction1;
-
-import java.util.List;
-
-/**
- * Retrieve the last tx id from list of log segments
- */
-public class GetLastTxIdFunction extends AbstractFunction1<List<LogSegmentMetadata>, Long> {
-
- public static final GetLastTxIdFunction INSTANCE = new GetLastTxIdFunction();
-
- private GetLastTxIdFunction() {}
-
- @Override
- public Long apply(List<LogSegmentMetadata> segmentList) {
- long lastTxId = DistributedLogConstants.INVALID_TXID;
- for (LogSegmentMetadata l : segmentList) {
- lastTxId = Math.max(lastTxId, l.getLastTxId());
- }
- return lastTxId;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetVersionedValueFunction.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetVersionedValueFunction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetVersionedValueFunction.java
deleted file mode 100644
index 4e7844c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetVersionedValueFunction.java
+++ /dev/null
@@ -1,39 +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.function;
-
-import com.twitter.distributedlog.LogSegmentMetadata;
-import org.apache.bookkeeper.versioning.Versioned;
-import scala.Function1;
-import scala.runtime.AbstractFunction1;
-
-import java.util.List;
-
-/**
- * Function to get the versioned value from {@link org.apache.bookkeeper.versioning.Versioned}
- */
-public class GetVersionedValueFunction<T> extends AbstractFunction1<Versioned<T>, T> {
-
- public static final Function1<Versioned<List<LogSegmentMetadata>>, List<LogSegmentMetadata>>
- GET_LOGSEGMENT_LIST_FUNC = new GetVersionedValueFunction<List<LogSegmentMetadata>>();
-
- @Override
- public T apply(Versioned<T> versionedValue) {
- return versionedValue.getValue();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/VoidFunctions.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/VoidFunctions.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/VoidFunctions.java
deleted file mode 100644
index e260482..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/VoidFunctions.java
+++ /dev/null
@@ -1,34 +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.function;
-
-import scala.runtime.AbstractFunction1;
-
-import java.util.List;
-
-public class VoidFunctions {
-
- public static final AbstractFunction1<List<Void>, Void> LIST_TO_VOID_FUNC =
- new AbstractFunction1<List<Void>, Void>() {
- @Override
- public Void apply(List<Void> list) {
- return null;
- }
- };
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/package-info.java
deleted file mode 100644
index 2da98dc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Common Functions for DistributedLog
- */
-package com.twitter.distributedlog.function;
[15/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/Tool.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/Tool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/Tool.java
deleted file mode 100644
index bb14066..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/Tool.java
+++ /dev/null
@@ -1,243 +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.tools;
-
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-/**
- * A Tool Framework
- */
-public abstract class Tool {
-
- /**
- * Interface of a command to run in a tool.
- */
- protected interface Command {
- String getName();
- String getDescription();
- int runCmd(String[] args) throws Exception;
- void printUsage();
- }
-
- /**
- * {@link org.apache.commons.cli.Options} based command.
- */
- protected abstract static class OptsCommand implements Command {
-
- /**
- * @return options used by this command.
- */
- protected abstract Options getOptions();
-
- /**
- * @return usage of this command.
- */
- protected String getUsage() {
- return cmdName + " [options]";
- }
-
- /**
- * Run given command line <i>commandLine</i>.
- *
- * @param commandLine
- * command line to run.
- * @return return code of this command.
- * @throws Exception
- */
- protected abstract int runCmd(CommandLine commandLine) throws Exception;
-
- protected String cmdName;
- protected String description;
-
- protected OptsCommand(String name, String description) {
- this.cmdName = name;
- this.description = description;
- }
-
- @Override
- public String getName() {
- return cmdName;
- }
-
- @Override
- public String getDescription() {
- return description;
- }
-
- @Override
- public int runCmd(String[] args) throws Exception {
- try {
- BasicParser parser = new BasicParser();
- CommandLine cmdline = parser.parse(getOptions(), args);
- return runCmd(cmdline);
- } catch (ParseException e) {
- printUsage();
- return -1;
- }
- }
-
- @Override
- public void printUsage() {
- HelpFormatter helpFormatter = new HelpFormatter();
- println(cmdName + ": " + getDescription());
- helpFormatter.printHelp(getUsage(), getOptions());
- }
- }
-
- public class HelpCommand implements Command {
-
- @Override
- public String getName() {
- return "help";
- }
-
- @Override
- public String getDescription() {
- return "describe the usage of this tool or its sub-commands.";
- }
-
- @Override
- public int runCmd(String[] args) throws Exception {
- if (args.length == 0) {
- printToolUsage();
- return -1;
- }
- String cmdName = args[0];
- Command command = commands.get(cmdName);
- if (null == command) {
- System.err.println("Unknown command " + cmdName);
- printToolUsage();
- return -1;
- }
- command.printUsage();
- println("");
- return 0;
- }
-
- @Override
- public void printUsage() {
- println(getName() + ": " + getDescription());
- println("");
- println("usage: " + getName() + " <command>");
- }
- }
-
- // Commands managed by a tool
- protected final Map<String, Command> commands =
- new TreeMap<String, Command>();
-
- protected Tool() {
- addCommand(new HelpCommand());
- }
-
- /**
- * @return tool name.
- */
- protected abstract String getName();
-
- /**
- * Add a command in this tool.
- *
- * @param command
- * command to run in this tool.
- */
- protected void addCommand(Command command) {
- commands.put(command.getName(), command);
- }
-
- /**
- * Print a message in this tool.
- *
- * @param msg
- * message to print
- */
- protected static void println(String msg) {
- System.out.println(msg);
- }
-
- /**
- * print tool usage.
- */
- protected void printToolUsage() {
- println("Usage: " + getName() + " <command>");
- println("");
- int maxKeyLength = 0;
- for (String key : commands.keySet()) {
- if (key.length() > maxKeyLength) {
- maxKeyLength = key.length();
- }
- }
- maxKeyLength += 2;
- for (Map.Entry<String, Command> entry : commands.entrySet()) {
- StringBuilder spacesBuilder = new StringBuilder();
- int numSpaces = maxKeyLength - entry.getKey().length();
- for (int i = 0; i < numSpaces; i++) {
- spacesBuilder.append(" ");
- }
- println("\t" + entry.getKey() + spacesBuilder.toString() + ": " + entry.getValue().getDescription());
- }
- println("");
- }
-
- public int run(String[] args) throws Exception {
- if (args.length <= 0) {
- printToolUsage();
- return -1;
- }
- String cmdName = args[0];
- Command cmd = commands.get(cmdName);
- if (null == cmd) {
- System.err.println("ERROR: Unknown command " + cmdName);
- printToolUsage();
- return -1;
- }
- // prepare new args
- String[] newArgs = new String[args.length - 1];
- System.arraycopy(args, 1, newArgs, 0, newArgs.length);
- return cmd.runCmd(newArgs);
- }
-
- public static void main(String args[]) {
- int rc = -1;
- if (args.length <= 0) {
- System.err.println("No tool to run.");
- System.err.println("");
- System.err.println("Usage : Tool <tool_class_name> <options>");
- System.exit(-1);
- }
- String toolClass = args[0];
- try {
- Tool tool = ReflectionUtils.newInstance(toolClass, Tool.class);
- String[] newArgs = new String[args.length - 1];
- System.arraycopy(args, 1, newArgs, 0, newArgs.length);
- rc = tool.run(newArgs);
- } catch (Throwable t) {
- System.err.println("Fail to run tool " + toolClass + " : ");
- t.printStackTrace();
- }
- System.exit(rc);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/package-info.java
deleted file mode 100644
index e2125bc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Tools for distributedlog
- */
-package com.twitter.distributedlog.tools;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Allocator.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Allocator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Allocator.java
deleted file mode 100644
index dcc3f58..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Allocator.java
+++ /dev/null
@@ -1,102 +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.util;
-
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.io.AsyncDeleteable;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.util.Future;
-
-import java.io.IOException;
-
-/**
- * A common interface to allocate <i>I</i> under transaction <i>T</i>.
- *
- * <h3>Usage Example</h3>
- *
- * Here is an example on demonstrating how `Allocator` works.
- *
- * <pre> {@code
- * Allocator<I, T, R> allocator = ...;
- *
- * // issue an allocate request
- * try {
- * allocator.allocate();
- * } catch (IOException ioe) {
- * // handle the exception
- * ...
- * return;
- * }
- *
- * // Start a transaction
- * final Transaction<T> txn = ...;
- *
- * // Try obtain object I
- * Future<I> tryObtainFuture = allocator.tryObtain(txn, new OpListener<I>() {
- * public void onCommit(I resource) {
- * // the obtain succeed, process with the resource
- * }
- * public void onAbort() {
- * // the obtain failed.
- * }
- * }).addFutureEventListener(new FutureEventListener() {
- * public void onSuccess(I resource) {
- * // the try obtain succeed. but the obtain has not been confirmed or aborted.
- * // execute the transaction to confirm if it could complete obtain
- * txn.execute();
- * }
- * public void onFailure(Throwable t) {
- * // handle the failure of try obtain
- * }
- * });
- *
- * }</pre>
- */
-public interface Allocator<I, T> extends AsyncCloseable, AsyncDeleteable {
-
- /**
- * Issue allocation request to allocate <i>I</i>.
- * The implementation should be non-blocking call.
- *
- * @throws IOException
- * if fail to request allocating a <i>I</i>.
- */
- void allocate() throws IOException;
-
- /**
- * Try obtaining an <i>I</i> in a given transaction <i>T</i>. The object obtained is tentative.
- * Whether the object is obtained or aborted is determined by the result of the execution. You could
- * register a listener under this `tryObtain` operation to know whether the object is obtained or
- * aborted.
- *
- * <p>
- * It is a typical two-phases operation on obtaining a resource from allocator.
- * The future returned by this method acts as a `prepare` operation, the resource is tentative obtained
- * from the allocator. The execution of the txn acts as a `commit` operation, the resource is confirmed
- * to be obtained by this transaction. <code>listener</code> is for the whole completion of the obtain.
- * <p>
- * <code>listener</code> is only triggered after `prepare` succeed. if `prepare` failed, no actions will
- * happen to the listener.
- *
- * @param txn
- * transaction.
- * @return future result returning <i>I</i> that would be obtained under transaction <code>txn</code>.
- */
- Future<I> tryObtain(Transaction<T> txn, OpListener<I> listener);
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/CommandLineUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/CommandLineUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/CommandLineUtils.java
deleted file mode 100644
index 95ef3e2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/CommandLineUtils.java
+++ /dev/null
@@ -1,56 +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.util;
-
-import com.google.common.base.Optional;
-import org.apache.commons.cli.CommandLine;
-
-/**
- * Utils to commandline
- */
-public class CommandLineUtils {
-
- public static Optional<String> getOptionalStringArg(CommandLine cmdline, String arg) {
- if (cmdline.hasOption(arg)) {
- return Optional.of(cmdline.getOptionValue(arg));
- } else {
- return Optional.absent();
- }
- }
-
- public static Optional<Boolean> getOptionalBooleanArg(CommandLine cmdline, String arg) {
- if (cmdline.hasOption(arg)) {
- return Optional.of(true);
- } else {
- return Optional.absent();
- }
- }
-
- public static Optional<Integer> getOptionalIntegerArg(CommandLine cmdline, String arg) throws IllegalArgumentException {
- try {
- if (cmdline.hasOption(arg)) {
- return Optional.of(Integer.parseInt(cmdline.getOptionValue(arg)));
- } else {
- return Optional.absent();
- }
- } catch (NumberFormatException ex) {
- throw new IllegalArgumentException(arg + " is not a number");
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/ConfUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/ConfUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/ConfUtils.java
deleted file mode 100644
index 46dd3b6..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/ConfUtils.java
+++ /dev/null
@@ -1,65 +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.util;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.config.ConcurrentConstConfiguration;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.commons.configuration.Configuration;
-
-import java.util.Iterator;
-
-public class ConfUtils {
-
- /**
- * Load configurations with prefixed <i>section</i> from source configuration <i>srcConf</i> into
- * target configuration <i>targetConf</i>.
- *
- * @param targetConf
- * Target Configuration
- * @param srcConf
- * Source Configuration
- * @param section
- * Section Key
- */
- public static void loadConfiguration(Configuration targetConf, Configuration srcConf, String section) {
- Iterator confKeys = srcConf.getKeys();
- while (confKeys.hasNext()) {
- Object keyObject = confKeys.next();
- if (!(keyObject instanceof String)) {
- continue;
- }
- String key = (String) keyObject;
- if (key.startsWith(section)) {
- targetConf.setProperty(key.substring(section.length()), srcConf.getProperty(key));
- }
- }
- }
-
- /**
- * Create const dynamic configuration based on distributedlog configuration.
- *
- * @param conf
- * static distributedlog configuration.
- * @return dynamic configuration
- */
- public static DynamicDistributedLogConfiguration getConstDynConf(DistributedLogConfiguration conf) {
- ConcurrentConstConfiguration constConf = new ConcurrentConstConfiguration(conf);
- return new DynamicDistributedLogConfiguration(constConf);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 2f9e091..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
+++ /dev/null
@@ -1,321 +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.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.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 {
-
- /**
- * Find the log segment whose transaction ids are not less than provided <code>transactionId</code>.
- *
- * @param segments
- * segments to search
- * @param transactionId
- * transaction id to find
- * @return the first log segment whose transaction ids are not less than <code>transactionId</code>.
- */
- public static int findLogSegmentNotLessThanTxnId(List<LogSegmentMetadata> segments,
- long transactionId) {
- int found = -1;
- for (int i = segments.size() - 1; i >= 0; i--) {
- LogSegmentMetadata segment = segments.get(i);
- if (segment.getFirstTxId() <= transactionId) {
- found = i;
- break;
- }
- }
- if (found <= -1) {
- return -1;
- }
- if (found == 0 && segments.get(0).getFirstTxId() == transactionId) {
- return 0;
- }
- LogSegmentMetadata foundSegment = segments.get(found);
- if (foundSegment.getFirstTxId() == transactionId) {
- for (int i = found - 1; i >= 0; i--) {
- LogSegmentMetadata segment = segments.get(i);
- if (segment.isInProgress()) {
- break;
- }
- if (segment.getLastTxId() < transactionId) {
- break;
- }
- found = i;
- }
- return found;
- } else {
- if (foundSegment.isInProgress()
- || found == segments.size() - 1) {
- return found;
- }
- if (foundSegment.getLastTxId() >= transactionId) {
- return found;
- }
- return found + 1;
- }
- }
-
- /**
- * Assign next log segment sequence number based on a decreasing list of log segments.
- *
- * @param segmentListDesc
- * a decreasing list of log segments
- * @return null if no log segments was assigned a sequence number in <code>segmentListDesc</code>.
- * otherwise, return next log segment sequence number
- */
- public static Long nextLogSegmentSequenceNumber(List<LogSegmentMetadata> segmentListDesc) {
- int lastAssignedLogSegmentIdx = -1;
- Long lastAssignedLogSegmentSeqNo = null;
- Long nextLogSegmentSeqNo = null;
-
- for (int i = 0; i < segmentListDesc.size(); i++) {
- LogSegmentMetadata metadata = segmentListDesc.get(i);
- if (LogSegmentMetadata.supportsLogSegmentSequenceNo(metadata.getVersion())) {
- lastAssignedLogSegmentSeqNo = metadata.getLogSegmentSequenceNumber();
- lastAssignedLogSegmentIdx = i;
- break;
- }
- }
-
- if (null != lastAssignedLogSegmentSeqNo) {
- // latest log segment is assigned with a sequence number, start with next sequence number
- nextLogSegmentSeqNo = lastAssignedLogSegmentSeqNo + lastAssignedLogSegmentIdx + 1;
- }
- return nextLogSegmentSeqNo;
- }
-
- /**
- * Compute the start sequence id for <code>segment</code>, based on previous segment list
- * <code>segmentListDesc</code>.
- *
- * @param logSegmentDescList
- * list of segments in descending order
- * @param segment
- * segment to compute start sequence id for
- * @return start sequence id
- */
- public static long computeStartSequenceId(List<LogSegmentMetadata> logSegmentDescList,
- LogSegmentMetadata segment)
- throws UnexpectedException {
- long startSequenceId = 0L;
- for (LogSegmentMetadata metadata : logSegmentDescList) {
- if (metadata.getLogSegmentSequenceNumber() >= segment.getLogSegmentSequenceNumber()) {
- continue;
- } else if (metadata.getLogSegmentSequenceNumber() < (segment.getLogSegmentSequenceNumber() - 1)) {
- break;
- }
- if (metadata.isInProgress()) {
- throw new UnexpectedException("Should not complete log segment " + segment.getLogSegmentSequenceNumber()
- + " since it's previous log segment is still inprogress : " + logSegmentDescList);
- }
- if (metadata.supportsSequenceId()) {
- startSequenceId = metadata.getStartSequenceId() + metadata.getRecordCount();
- }
- }
- return startSequenceId;
- }
-
- /**
- * Deserialize log segment sequence number for bytes <code>data</code>.
- *
- * @param data
- * byte representation of log segment sequence number
- * @return log segment sequence number
- * @throws NumberFormatException if the bytes aren't valid
- */
- public static long deserializeLogSegmentSequenceNumber(byte[] data) {
- String seqNoStr = new String(data, UTF_8);
- return Long.parseLong(seqNoStr);
- }
-
- /**
- * Serilize log segment sequence number <code>logSegmentSeqNo</code> into bytes.
- *
- * @param logSegmentSeqNo
- * log segment sequence number
- * @return byte representation of log segment sequence number
- */
- public static byte[] serializeLogSegmentSequenceNumber(long logSegmentSeqNo) {
- return Long.toString(logSegmentSeqNo).getBytes(UTF_8);
- }
-
- /**
- * Deserialize log record transaction id for bytes <code>data</code>.
- *
- * @param data
- * byte representation of log record transaction id
- * @return log record transaction id
- * @throws NumberFormatException if the bytes aren't valid
- */
- public static long deserializeTransactionId(byte[] data) {
- String seqNoStr = new String(data, UTF_8);
- return Long.parseLong(seqNoStr);
- }
-
- /**
- * Serilize log record transaction id <code>transactionId</code> into bytes.
- *
- * @param transactionId
- * log record transaction id
- * @return byte representation of log record transaction id.
- */
- public static byte[] serializeTransactionId(long transactionId) {
- return Long.toString(transactionId).getBytes(UTF_8);
- }
-
- /**
- * Serialize log segment id into bytes.
- *
- * @param logSegmentId
- * log segment id
- * @return bytes representation of log segment id
- */
- public static byte[] logSegmentId2Bytes(long logSegmentId) {
- return Long.toString(logSegmentId).getBytes(UTF_8);
- }
-
- /**
- * Deserialize bytes into log segment id.
- *
- * @param data
- * bytes representation of log segment id
- * @return log segment id
- */
- 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");
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FailpointUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FailpointUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FailpointUtils.java
deleted file mode 100644
index 64101b3..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FailpointUtils.java
+++ /dev/null
@@ -1,134 +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.util;
-
-import java.io.IOException;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class FailpointUtils {
- static final Logger logger = LoggerFactory.getLogger(FailpointUtils.class);
-
- public enum FailPointName {
- FP_StartLogSegmentBeforeLedgerCreate,
- FP_StartLogSegmentAfterLedgerCreate,
- FP_StartLogSegmentAfterInProgressCreate,
- FP_StartLogSegmentOnAssignLogSegmentSequenceNumber,
- FP_FinalizeLedgerBeforeDelete,
- FP_TransmitBeforeAddEntry,
- FP_TransmitComplete,
- FP_WriteInternalLostLock,
- FP_TransmitFailGetBuffer,
- FP_LockUnlockCleanup,
- FP_LockTryCloseRaceCondition,
- FP_LockTryAcquire,
- FP_ZooKeeperConnectionLoss,
- FP_RecoverIncompleteLogSegments,
- FP_LogWriterIssuePending,
- }
-
- public static interface FailPointAction {
- boolean checkFailPoint() throws IOException;
- boolean checkFailPointNoThrow();
- }
-
- public static abstract class AbstractFailPointAction implements FailPointAction {
- @Override
- public boolean checkFailPointNoThrow() {
- try {
- return checkFailPoint();
- } catch (IOException ex) {
- logger.error("failpoint action raised unexpected exception");
- return true;
- }
- }
- }
-
- public static final FailPointAction DEFAULT_ACTION = new AbstractFailPointAction() {
- @Override
- public boolean checkFailPoint() throws IOException {
- return true;
- }
- };
-
- public static final FailPointAction THROW_ACTION = new AbstractFailPointAction() {
- @Override
- public boolean checkFailPoint() throws IOException {
- throw new IOException("Throw ioexception for failure point");
- }
- };
-
- public enum FailPointActions {
- FailPointAction_Default,
- FailPointAction_Throw
- }
-
- static ConcurrentHashMap<FailPointName, FailPointAction> failPointState =
- new ConcurrentHashMap<FailPointName, FailPointAction>();
-
- public static void setFailpoint(FailPointName failpoint, FailPointActions action) {
- FailPointAction fpAction = null;
- switch (action) {
- case FailPointAction_Default:
- fpAction = DEFAULT_ACTION;
- break;
- case FailPointAction_Throw:
- fpAction = THROW_ACTION;
- break;
- default:
- break;
- }
- setFailpoint(failpoint, fpAction);
- }
-
- public static void setFailpoint(FailPointName failpoint, FailPointAction action) {
- if (null != action) {
- failPointState.put(failpoint, action);
- }
- }
-
- public static void removeFailpoint(FailPointName failpoint) {
- failPointState.remove(failpoint);
- }
-
- public static boolean checkFailPoint(FailPointName failPoint) throws IOException {
- FailPointAction action = failPointState.get(failPoint);
-
- if (action == null) {
- return false;
- }
-
- try {
- return action.checkFailPoint();
- } catch (IOException ioe) {
- throw new IOException("Induced Exception at:" + failPoint, ioe);
- }
- }
-
- public static boolean checkFailPointNoThrow(FailPointName failPoint) {
- FailPointAction action = failPointState.get(failPoint);
-
- if (action == null) {
- return false;
- }
-
- return action.checkFailPointNoThrow();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
----------------------------------------------------------------------
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
deleted file mode 100644
index f206a25..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
+++ /dev/null
@@ -1,534 +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.util;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.stats.OpStatsListener;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureCancelledException;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import com.twitter.util.Try;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import javax.annotation.Nullable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Utilities to process future
- */
-public class FutureUtils {
-
- private static final Logger logger = LoggerFactory.getLogger(FutureUtils.class);
-
- public static class OrderedFutureEventListener<R>
- implements FutureEventListener<R> {
-
- public static <R> OrderedFutureEventListener<R> of(
- FutureEventListener<R> listener,
- OrderedScheduler scheduler,
- Object key) {
- return new OrderedFutureEventListener<R>(scheduler, key, listener);
- }
-
- private final OrderedScheduler scheduler;
- private final Object key;
- private final FutureEventListener<R> listener;
-
- private OrderedFutureEventListener(OrderedScheduler scheduler,
- Object key,
- FutureEventListener<R> listener) {
- this.scheduler = scheduler;
- this.key = key;
- this.listener = listener;
- }
-
- @Override
- public void onSuccess(final R value) {
- scheduler.submit(key, new Runnable() {
- @Override
- public void run() {
- listener.onSuccess(value);
- }
- });
- }
-
- @Override
- public void onFailure(final Throwable cause) {
- scheduler.submit(key, new Runnable() {
- @Override
- public void run() {
- listener.onFailure(cause);
- }
- });
- }
- }
-
- public static class FutureEventListenerRunnable<R>
- implements FutureEventListener<R> {
-
- public static <R> FutureEventListenerRunnable<R> of(
- FutureEventListener<R> listener,
- ExecutorService executorService) {
- return new FutureEventListenerRunnable<R>(executorService, listener);
- }
-
- private final ExecutorService executorService;
- private final FutureEventListener<R> listener;
-
- private FutureEventListenerRunnable(ExecutorService executorService,
- FutureEventListener<R> listener) {
- this.executorService = executorService;
- this.listener = listener;
- }
-
- @Override
- public void onSuccess(final R value) {
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- listener.onSuccess(value);
- }
- });
- }
-
- @Override
- public void onFailure(final Throwable cause) {
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- listener.onFailure(cause);
- }
- });
- }
- }
-
- private static class ListFutureProcessor<T, R>
- extends Function<Throwable, BoxedUnit>
- implements FutureEventListener<R>, Runnable {
-
- private volatile boolean interrupted = false;
- private final Iterator<T> itemsIter;
- private final Function<T, Future<R>> processFunc;
- private final Promise<List<R>> promise;
- private final List<R> results;
- private final ExecutorService callbackExecutor;
-
- ListFutureProcessor(List<T> items,
- Function<T, Future<R>> processFunc,
- ExecutorService callbackExecutor) {
- this.itemsIter = items.iterator();
- this.processFunc = processFunc;
- this.promise = new Promise<List<R>>();
- this.promise.setInterruptHandler(this);
- this.results = new ArrayList<R>();
- this.callbackExecutor = callbackExecutor;
- }
-
- @Override
- public BoxedUnit apply(Throwable cause) {
- interrupted = true;
- return BoxedUnit.UNIT;
- }
-
- @Override
- public void onSuccess(R value) {
- results.add(value);
- if (null == callbackExecutor) {
- run();
- } else {
- callbackExecutor.submit(this);
- }
- }
-
- @Override
- public void onFailure(final Throwable cause) {
- if (null == callbackExecutor) {
- promise.setException(cause);
- } else {
- callbackExecutor.submit(new Runnable() {
- @Override
- public void run() {
- promise.setException(cause);
- }
- });
- }
- }
-
- @Override
- public void run() {
- if (interrupted) {
- logger.debug("ListFutureProcessor is interrupted.");
- return;
- }
- if (!itemsIter.hasNext()) {
- promise.setValue(results);
- return;
- }
- processFunc.apply(itemsIter.next()).addEventListener(this);
- }
- }
-
- /**
- * Process the list of items one by one using the process function <i>processFunc</i>.
- * The process will be stopped immediately if it fails on processing any one.
- *
- * @param collection list of items
- * @param processFunc process function
- * @param callbackExecutor executor to process the item
- * @return future presents the list of processed results
- */
- public static <T, R> Future<List<R>> processList(List<T> collection,
- Function<T, Future<R>> processFunc,
- @Nullable ExecutorService callbackExecutor) {
- ListFutureProcessor<T, R> processor =
- new ListFutureProcessor<T, R>(collection, processFunc, callbackExecutor);
- if (null != callbackExecutor) {
- callbackExecutor.submit(processor);
- } else {
- processor.run();
- }
- return processor.promise;
- }
-
- /**
- * Add a event listener over <i>result</i> for collecting the operation stats.
- *
- * @param result result to listen on
- * @param opStatsLogger stats logger to record operations stats
- * @param stopwatch stop watch to time operation
- * @param <T>
- * @return result after registered the event listener
- */
- public static <T> Future<T> stats(Future<T> result,
- OpStatsLogger opStatsLogger,
- Stopwatch stopwatch) {
- return result.addEventListener(new OpStatsListener<T>(opStatsLogger, stopwatch));
- }
-
- /**
- * Await for the result of the future and thrown bk related exceptions.
- *
- * @param result future to wait for
- * @return the result of future
- * @throws BKException when exceptions are thrown by the future. If there is unkown exceptions
- * thrown from the future, the exceptions will be wrapped into
- * {@link org.apache.bookkeeper.client.BKException.BKUnexpectedConditionException}.
- */
- public static <T> T bkResult(Future<T> result) throws BKException {
- try {
- return Await.result(result);
- } catch (BKException bke) {
- throw bke;
- } catch (InterruptedException ie) {
- throw BKException.create(BKException.Code.InterruptedException);
- } catch (Exception e) {
- logger.warn("Encountered unexpected exception on waiting bookkeeper results : ", e);
- throw BKException.create(BKException.Code.UnexpectedConditionException);
- }
- }
-
- /**
- * Return the bk exception return code for a <i>throwable</i>.
- *
- * @param throwable the cause of the exception
- * @return the bk exception return code. if the exception isn't bk exceptions,
- * it would return {@link BKException.Code#UnexpectedConditionException}.
- */
- public static int bkResultCode(Throwable throwable) {
- if (throwable instanceof BKException) {
- return ((BKException)throwable).getCode();
- }
- return BKException.Code.UnexpectedConditionException;
- }
-
- /**
- * Wait for the result until it completes.
- *
- * @param result result to wait
- * @return the result
- * @throws IOException when encountered exceptions on the result
- */
- public static <T> T result(Future<T> result) throws IOException {
- return result(result, Duration.Top());
- }
-
- /**
- * Wait for the result for a given <i>duration</i>.
- * <p>If the result is not ready within `duration`, an IOException will thrown wrapping with
- * corresponding {@link com.twitter.util.TimeoutException}.
- *
- * @param result result to wait
- * @param duration duration to wait
- * @return the result
- * @throws IOException when encountered exceptions on the result or waiting for the result.
- */
- public static <T> T result(Future<T> result, Duration duration)
- throws IOException {
- try {
- return Await.result(result, duration);
- } catch (KeeperException ke) {
- throw new ZKException("Encountered zookeeper exception on waiting result", ke);
- } catch (BKException bke) {
- throw new BKTransmitException("Encountered bookkeeper exception on waiting result", bke.getCode());
- } catch (IOException ioe) {
- throw ioe;
- } catch (InterruptedException ie) {
- throw new DLInterruptedException("Interrupted on waiting result", ie);
- } catch (Exception e) {
- throw new IOException("Encountered exception on waiting result", e);
- }
- }
-
- /**
- * Wait for the result of a lock operation.
- *
- * @param result result to wait
- * @param lockPath path of the lock
- * @return the result
- * @throws LockingException when encountered exceptions on the result of lock operation
- */
- public static <T> T lockResult(Future<T> result, String lockPath) throws LockingException {
- try {
- return Await.result(result);
- } catch (LockingException le) {
- throw le;
- } catch (Exception e) {
- throw new LockingException(lockPath, "Encountered exception on locking ", e);
- }
- }
-
- /**
- * Convert the <i>throwable</i> to zookeeper related exceptions.
- *
- * @param throwable cause
- * @param path zookeeper path
- * @return zookeeper related exceptions
- */
- public static Throwable zkException(Throwable throwable, String path) {
- if (throwable instanceof KeeperException) {
- return new ZKException("Encountered zookeeper exception on " + path, (KeeperException) throwable);
- } else if (throwable instanceof ZooKeeperClient.ZooKeeperConnectionException) {
- return new ZKException("Encountered zookeeper connection loss on " + path,
- KeeperException.Code.CONNECTIONLOSS);
- } else if (throwable instanceof InterruptedException) {
- return new DLInterruptedException("Interrupted on operating " + path, throwable);
- } else {
- return new UnexpectedException("Encountered unexpected exception on operatiing " + path, throwable);
- }
- }
-
- /**
- * Cancel the future. It would interrupt the future.
- *
- * @param future future to cancel
- */
- public static <T> void cancel(Future<T> future) {
- future.raise(new FutureCancelledException());
- }
-
- /**
- * Raise an exception to the <i>promise</i> within a given <i>timeout</i> period.
- * If the promise has been satisfied before raising, it won't change the state of the promise.
- *
- * @param promise promise to raise exception
- * @param timeout timeout period
- * @param unit timeout period unit
- * @param cause cause to raise
- * @param scheduler scheduler to execute raising exception
- * @param key the submit key used by the scheduler
- * @return the promise applied with the raise logic
- */
- public static <T> Promise<T> within(final Promise<T> promise,
- final long timeout,
- final TimeUnit unit,
- final Throwable cause,
- final OrderedScheduler scheduler,
- final Object key) {
- if (timeout < DistributedLogConstants.FUTURE_TIMEOUT_IMMEDIATE || promise.isDefined()) {
- return promise;
- }
- // schedule a timeout to raise timeout exception
- final java.util.concurrent.ScheduledFuture<?> task = scheduler.schedule(key, new Runnable() {
- @Override
- public void run() {
- if (!promise.isDefined() && FutureUtils.setException(promise, cause)) {
- logger.info("Raise exception", cause);
- }
- }
- }, timeout, unit);
- // when the promise is satisfied, cancel the timeout task
- promise.respond(new AbstractFunction1<Try<T>, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Try<T> value) {
- if (!task.cancel(true)) {
- logger.debug("Failed to cancel the timeout task");
- }
- return BoxedUnit.UNIT;
- }
- });
- return promise;
- }
-
- /**
- * Satisfy the <i>promise</i> with provide value in an ordered scheduler.
- * <p>If the promise was already satisfied, nothing will be changed.
- *
- * @param promise promise to satisfy
- * @param value value to satisfy
- * @param scheduler scheduler to satisfy the promise with provided value
- * @param key the submit key of the ordered scheduler
- */
- public static <T> void setValue(final Promise<T> promise,
- final T value,
- OrderedScheduler scheduler,
- Object key) {
- scheduler.submit(key, new Runnable() {
- @Override
- public void run() {
- setValue(promise, value);
- }
- });
- }
-
- /**
- * Satisfy the <i>promise</i> with provide value.
- * <p>If the promise was already satisfied, nothing will be changed.
- *
- * @param promise promise to satisfy
- * @param value value to satisfy
- * @return true if successfully satisfy the future. false if the promise has been satisfied.
- */
- public static <T> boolean setValue(Promise<T> promise, T value) {
- boolean success = promise.updateIfEmpty(new Return<T>(value));
- if (!success) {
- logger.info("Result set multiple times. Value = '{}', New = 'Return({})'",
- promise.poll(), value);
- }
- return success;
- }
-
- /**
- * Satisfy the <i>promise</i> with provided <i>cause</i> in an ordered scheduler.
- *
- * @param promise promise to satisfy
- * @param throwable cause to satisfy
- * @param scheduler the scheduler to satisfy the promise
- * @param key submit key of the ordered scheduler
- */
- public static <T> void setException(final Promise<T> promise,
- final Throwable cause,
- OrderedScheduler scheduler,
- Object key) {
- scheduler.submit(key, new Runnable() {
- @Override
- public void run() {
- setException(promise, cause);
- }
- });
- }
-
- /**
- * Satisfy the <i>promise</i> with provided <i>cause</i>.
- *
- * @param promise promise to satisfy
- * @param cause cause to satisfy
- * @return true if successfully satisfy the future. false if the promise has been satisfied.
- */
- public static <T> boolean setException(Promise<T> promise, Throwable cause) {
- boolean success = promise.updateIfEmpty(new Throw<T>(cause));
- if (!success) {
- logger.info("Result set multiple times. Value = '{}', New = 'Throw({})'",
- promise.poll(), cause);
- }
- return success;
- }
-
- /**
- * Ignore exception from the <i>future</i>.
- *
- * @param future the original future
- * @return a transformed future ignores exceptions
- */
- public static <T> Promise<Void> ignore(Future<T> future) {
- return ignore(future, null);
- }
-
- /**
- * Ignore exception from the <i>future</i> and log <i>errorMsg</i> on exceptions
- *
- * @param future the original future
- * @param errorMsg the error message to log on exceptions
- * @return a transformed future ignores exceptions
- */
- public static <T> Promise<Void> ignore(Future<T> future, final String errorMsg) {
- final Promise<Void> promise = new Promise<Void>();
- future.addEventListener(new FutureEventListener<T>() {
- @Override
- public void onSuccess(T value) {
- setValue(promise, null);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- if (null != errorMsg) {
- logger.error(errorMsg, cause);
- }
- setValue(promise, null);
- }
- });
- return promise;
- }
-
- /**
- * Create transmit exception from transmit result.
- *
- * @param transmitResult
- * transmit result (basically bk exception code)
- * @return transmit exception
- */
- public static BKTransmitException transmitException(int transmitResult) {
- return new BKTransmitException("Failed to write to bookkeeper; Error is ("
- + transmitResult + ") "
- + BKException.getMessage(transmitResult), transmitResult);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredFuturePool.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredFuturePool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredFuturePool.java
deleted file mode 100644
index e06023e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredFuturePool.java
+++ /dev/null
@@ -1,131 +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.util;
-
-import com.google.common.base.Stopwatch;
-
-import com.twitter.util.FuturePool;
-import com.twitter.util.FuturePool$;
-import com.twitter.util.Future;
-
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import scala.runtime.BoxedUnit;
-import scala.Function0;
-
-/**
- * {@link FuturePool} with exposed stats. This class is exposing following stats for helping understanding
- * the healthy of this thread pool executor.
- * <h3>Metrics</h3>
- * Stats are only exposed when <code>traceTaskExecution</code> is true.
- * <ul>
- * <li>task_pending_time: opstats. measuring the characteristics about the time that tasks spent on waiting
- * being executed.
- * <li>task_execution_time: opstats. measuring the characteristics about the time that tasks spent on executing.
- * <li>task_enqueue_time: opstats. measuring the characteristics about the time that tasks spent on submitting.
- * <li>tasks_pending: gauge. how many tasks are pending in this future pool.
- * </ul>
- */
-public class MonitoredFuturePool implements FuturePool {
- static final Logger LOG = LoggerFactory.getLogger(MonitoredFuturePool.class);
-
- private final FuturePool futurePool;
-
- private final StatsLogger statsLogger;
- private final OpStatsLogger taskPendingTime;
- private final OpStatsLogger taskExecutionTime;
- private final OpStatsLogger taskEnqueueTime;
- private final Counter taskPendingCounter;
-
- private final boolean traceTaskExecution;
- private final long traceTaskExecutionWarnTimeUs;
-
- class TimedFunction0<T> extends com.twitter.util.Function0<T> {
- private final Function0<T> function0;
- private Stopwatch pendingStopwatch = Stopwatch.createStarted();
-
- TimedFunction0(Function0<T> function0) {
- this.function0 = function0;
- this.pendingStopwatch = Stopwatch.createStarted();
- }
-
- @Override
- public T apply() {
- taskPendingTime.registerSuccessfulEvent(pendingStopwatch.elapsed(TimeUnit.MICROSECONDS));
- Stopwatch executionStopwatch = Stopwatch.createStarted();
- T result = function0.apply();
- taskExecutionTime.registerSuccessfulEvent(executionStopwatch.elapsed(TimeUnit.MICROSECONDS));
- long elapsed = executionStopwatch.elapsed(TimeUnit.MICROSECONDS);
- if (elapsed > traceTaskExecutionWarnTimeUs) {
- LOG.info("{} took too long {} microseconds", function0.toString(), elapsed);
- }
- return result;
- }
- }
-
- /**
- * Create a future pool with stats exposed.
- *
- * @param futurePool underlying future pool to execute futures
- * @param statsLogger stats logger to receive exposed stats
- * @param traceTaskExecution flag to enable/disable exposing stats about task execution
- * @param traceTaskExecutionWarnTimeUs flag to enable/disable logging slow tasks
- * whose execution time is above this value
- */
- public MonitoredFuturePool(FuturePool futurePool,
- StatsLogger statsLogger,
- boolean traceTaskExecution,
- long traceTaskExecutionWarnTimeUs) {
- this.futurePool = futurePool;
- this.traceTaskExecution = traceTaskExecution;
- this.traceTaskExecutionWarnTimeUs = traceTaskExecutionWarnTimeUs;
- this.statsLogger = statsLogger;
- this.taskPendingTime = statsLogger.getOpStatsLogger("task_pending_time");
- this.taskExecutionTime = statsLogger.getOpStatsLogger("task_execution_time");
- this.taskEnqueueTime = statsLogger.getOpStatsLogger("task_enqueue_time");
- this.taskPendingCounter = statsLogger.getCounter("tasks_pending");
- }
-
- @Override
- public <T> Future<T> apply(Function0<T> function0) {
- if (traceTaskExecution) {
- taskPendingCounter.inc();
- Stopwatch taskEnqueueStopwatch = Stopwatch.createStarted();
- Future<T> futureResult = futurePool.apply(new TimedFunction0<T>(function0));
- taskEnqueueTime.registerSuccessfulEvent(taskEnqueueStopwatch.elapsed(TimeUnit.MICROSECONDS));
- futureResult.ensure(new com.twitter.util.Function0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- taskPendingCounter.dec();
- return null;
- }
- });
- return futureResult;
- } else {
- return futurePool.apply(function0);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java
deleted file mode 100644
index 75223f2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredScheduledThreadPoolExecutor.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.util;
-
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.MathUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-
-/**
- * {@link ScheduledThreadPoolExecutor} with exposed stats. This class is exposing following stats for
- * helping understanding the healthy of this thread pool executor.
- * <h3>Metrics</h3>
- * <ul>
- * <li>pending_tasks: gauge. how many tasks are pending in this executor.
- * <li>completed_tasks: gauge. how many tasks are completed in this executor.
- * <li>total_tasks: gauge. how many tasks are submitted to this executor.
- * <li>task_pending_time: opstats. measuring the characteristics about the time that tasks spent on
- * waiting being executed.
- * <li>task_execution_time: opstats. measuring the characteristics about the time that tasks spent on
- * executing.
- * </ul>
- */
-public class MonitoredScheduledThreadPoolExecutor extends ScheduledThreadPoolExecutor {
- static final Logger LOG = LoggerFactory.getLogger(MonitoredScheduledThreadPoolExecutor.class);
-
- private class TimedRunnable implements Runnable {
-
- final Runnable runnable;
- final long enqueueNanos;
-
- TimedRunnable(Runnable runnable) {
- this.runnable = runnable;
- this.enqueueNanos = MathUtils.nowInNano();
- }
-
- @Override
- public void run() {
- long startNanos = MathUtils.nowInNano();
- long pendingMicros = TimeUnit.NANOSECONDS.toMicros(startNanos - enqueueNanos);
- taskPendingStats.registerSuccessfulEvent(pendingMicros);
- try {
- runnable.run();
- } finally {
- long executionMicros = TimeUnit.NANOSECONDS.toMicros(MathUtils.nowInNano() - startNanos);
- taskExecutionStats.registerSuccessfulEvent(executionMicros);
- }
- }
-
- @Override
- public String toString() {
- return runnable.toString();
- }
-
- @Override
- public int hashCode() {
- return runnable.hashCode();
- }
- }
-
- private class TimedCallable<T> implements Callable<T> {
-
- final Callable<T> task;
- final long enqueueNanos;
-
- TimedCallable(Callable<T> task) {
- this.task = task;
- this.enqueueNanos = MathUtils.nowInNano();
- }
-
- @Override
- public T call() throws Exception {
- long startNanos = MathUtils.nowInNano();
- long pendingMicros = TimeUnit.NANOSECONDS.toMicros(startNanos - enqueueNanos);
- taskPendingStats.registerSuccessfulEvent(pendingMicros);
- try {
- return task.call();
- } finally {
- long executionMicros = TimeUnit.NANOSECONDS.toMicros(MathUtils.nowInNano() - startNanos);
- taskExecutionStats.registerSuccessfulEvent(executionMicros);
- }
- }
- }
-
- protected final boolean traceTaskExecution;
- protected final OpStatsLogger taskExecutionStats;
- protected final OpStatsLogger taskPendingStats;
- protected final StatsLogger statsLogger;
- // Gauges and their labels
- private static final String pendingTasksGaugeLabel = "pending_tasks";
- private final Gauge<Number> pendingTasksGauge;
- private static final String completedTasksGaugeLabel = "completed_tasks";
- protected final Gauge<Number> completedTasksGauge;
- private static final String totalTasksGaugeLabel = "total_tasks";
- protected final Gauge<Number> totalTasksGauge;
-
- public MonitoredScheduledThreadPoolExecutor(int corePoolSize,
- ThreadFactory threadFactory,
- StatsLogger statsLogger,
- boolean traceTaskExecution) {
- super(corePoolSize, threadFactory);
- this.traceTaskExecution = traceTaskExecution;
- this.statsLogger = statsLogger;
- this.taskPendingStats = this.statsLogger.getOpStatsLogger("task_pending_time");
- this.taskExecutionStats = this.statsLogger.getOpStatsLogger("task_execution_time");
- this.pendingTasksGauge = new Gauge<Number>() {
- @Override
- public Number getDefaultValue() {
- return 0;
- }
-
- @Override
- public Number getSample() {
- return getQueue().size();
- }
- };
- this.completedTasksGauge = new Gauge<Number>() {
- @Override
- public Number getDefaultValue() {
- return 0;
- }
-
- @Override
- public Number getSample() {
- return getCompletedTaskCount();
- }
- };
- this.totalTasksGauge = new Gauge<Number>() {
- @Override
- public Number getDefaultValue() {
- return 0;
- }
-
- @Override
- public Number getSample() {
- return getTaskCount();
- }
- };
-
- // outstanding tasks
- this.statsLogger.registerGauge(pendingTasksGaugeLabel, pendingTasksGauge);
- // completed tasks
- this.statsLogger.registerGauge(completedTasksGaugeLabel, completedTasksGauge);
- // total tasks
- this.statsLogger.registerGauge(totalTasksGaugeLabel, pendingTasksGauge);
- }
-
- private Runnable timedRunnable(Runnable r) {
- return traceTaskExecution ? new TimedRunnable(r) : r;
- }
-
- private <T> Callable<T> timedCallable(Callable<T> task) {
- return traceTaskExecution ? new TimedCallable<T>(task) : task;
- }
-
- @Override
- public Future<?> submit(Runnable task) {
- return super.submit(timedRunnable(task));
- }
-
- @Override
- public <T> Future<T> submit(Runnable task, T result) {
- return super.submit(timedRunnable(task), result);
- }
-
- @Override
- public <T> Future<T> submit(Callable<T> task) {
- return super.submit(timedCallable(task));
- }
-
- @Override
- protected void afterExecute(Runnable r, Throwable t) {
- super.afterExecute(r, t);
- Throwable hiddenThrowable = extractThrowable(r);
- if (hiddenThrowable != null)
- logAndHandle(hiddenThrowable, true);
-
- // The executor re-throws exceptions thrown by the task to the uncaught exception handler
- // so we don't need to pass the exception to the handler explicitly
- if (null != t) {
- logAndHandle(t, false);
- }
- }
-
- /**
- * The executor re-throws exceptions thrown by the task to the uncaught exception handler
- * so we only need to do anything if uncaught exception handler has not been se
- */
- private void logAndHandle(Throwable t, boolean passToHandler) {
- if (Thread.getDefaultUncaughtExceptionHandler() == null) {
- LOG.error("Unhandled exception on thread {}", Thread.currentThread().getName(), t);
- }
- else {
- LOG.info("Unhandled exception on thread {}", Thread.currentThread().getName(), t);
- if (passToHandler) {
- Thread.getDefaultUncaughtExceptionHandler().uncaughtException(Thread.currentThread(), t);
- }
- }
- }
-
-
- /**
- * Extract the exception (throwable) inside the ScheduledFutureTask
- * @param runnable - The runable that was executed
- * @return exception enclosed in the Runnable if any; null otherwise
- */
- private Throwable extractThrowable(Runnable runnable) {
- // Check for exceptions wrapped by FutureTask.
- // We do this by calling get(), which will cause it to throw any saved exception.
- // Check for isDone to prevent blocking
- if ((runnable instanceof Future<?>) && ((Future<?>) runnable).isDone()) {
- try {
- ((Future<?>) runnable).get();
- } catch (CancellationException e) {
- LOG.debug("Task {} cancelled", runnable, e.getCause());
- } catch (InterruptedException e) {
- LOG.debug("Task {} was interrupted", runnable, e);
- } catch (ExecutionException e) {
- return e.getCause();
- }
- }
-
- return null;
- }
-
- void unregisterGauges() {
- this.statsLogger.unregisterGauge(pendingTasksGaugeLabel, pendingTasksGauge);
- this.statsLogger.unregisterGauge(completedTasksGaugeLabel, completedTasksGauge);
- this.statsLogger.unregisterGauge(totalTasksGaugeLabel, totalTasksGauge);
- }
-
-}
[45/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetRoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetRoutingService.java
deleted file mode 100644
index 19ccfc4..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetRoutingService.java
+++ /dev/null
@@ -1,274 +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.client.routing;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Routing Service based on a given {@link com.twitter.common.zookeeper.ServerSet}.
- */
-class ServerSetRoutingService extends Thread implements RoutingService {
-
- private static final Logger logger = LoggerFactory.getLogger(ServerSetRoutingService.class);
-
- static ServerSetRoutingServiceBuilder newServerSetRoutingServiceBuilder() {
- return new ServerSetRoutingServiceBuilder();
- }
-
- /**
- * Builder to build {@link com.twitter.common.zookeeper.ServerSet} based routing service.
- */
- static class ServerSetRoutingServiceBuilder implements RoutingService.Builder {
-
- private ServerSetWatcher serverSetWatcher;
-
- private ServerSetRoutingServiceBuilder() {}
-
- public ServerSetRoutingServiceBuilder serverSetWatcher(ServerSetWatcher serverSetWatcher) {
- this.serverSetWatcher = serverSetWatcher;
- return this;
- }
-
- @Override
- public Builder statsReceiver(StatsReceiver statsReceiver) {
- return this;
- }
-
- @Override
- public RoutingService build() {
- checkNotNull(serverSetWatcher, "No serverset watcher provided.");
- return new ServerSetRoutingService(this.serverSetWatcher);
- }
- }
-
- private static class HostComparator implements Comparator<SocketAddress> {
-
- private static final HostComparator INSTANCE = new HostComparator();
-
- @Override
- public int compare(SocketAddress o1, SocketAddress o2) {
- return o1.toString().compareTo(o2.toString());
- }
- }
-
- private final ServerSetWatcher serverSetWatcher;
-
- private final Set<SocketAddress> hostSet = new HashSet<SocketAddress>();
- private List<SocketAddress> hostList = new ArrayList<SocketAddress>();
- private final HashFunction hasher = Hashing.md5();
-
- // Server Set Changes
- private final AtomicReference<ImmutableSet<DLSocketAddress>> serverSetChange =
- new AtomicReference<ImmutableSet<DLSocketAddress>>(null);
- private final CountDownLatch changeLatch = new CountDownLatch(1);
-
- // Listeners
- protected final CopyOnWriteArraySet<RoutingListener> listeners =
- new CopyOnWriteArraySet<RoutingListener>();
-
- ServerSetRoutingService(ServerSetWatcher serverSetWatcher) {
- super("ServerSetRoutingService");
- this.serverSetWatcher = serverSetWatcher;
- }
-
- @Override
- public Set<SocketAddress> getHosts() {
- synchronized (hostSet) {
- return ImmutableSet.copyOf(hostSet);
- }
- }
-
- @Override
- public void startService() {
- start();
- try {
- if (!changeLatch.await(1, TimeUnit.MINUTES)) {
- logger.warn("No serverset change received in 1 minute.");
- }
- } catch (InterruptedException e) {
- logger.warn("Interrupted waiting first serverset change : ", e);
- }
- logger.info("{} Routing Service Started.", getClass().getSimpleName());
- }
-
- @Override
- public void stopService() {
- Thread.currentThread().interrupt();
- try {
- join();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- logger.warn("Interrupted on waiting serverset routing service to finish : ", e);
- }
- logger.info("{} Routing Service Stopped.", getClass().getSimpleName());
- }
-
- @Override
- public RoutingService registerListener(RoutingListener listener) {
- listeners.add(listener);
- return this;
- }
-
- @Override
- public RoutingService unregisterListener(RoutingListener listener) {
- listeners.remove(listener);
- return this;
- }
-
- @Override
- public SocketAddress getHost(String key, RoutingContext rContext)
- throws NoBrokersAvailableException {
- SocketAddress address = null;
- synchronized (hostSet) {
- if (0 != hostList.size()) {
- int hashCode = hasher.hashUnencodedChars(key).asInt();
- int hostId = signSafeMod(hashCode, hostList.size());
- address = hostList.get(hostId);
- if (rContext.isTriedHost(address)) {
- ArrayList<SocketAddress> newList = new ArrayList<SocketAddress>(hostList);
- newList.remove(hostId);
- // pickup a new host by rehashing it.
- hostId = signSafeMod(hashCode, newList.size());
- address = newList.get(hostId);
- int i = hostId;
- while (rContext.isTriedHost(address)) {
- i = (i + 1) % newList.size();
- if (i == hostId) {
- address = null;
- break;
- }
- address = newList.get(i);
- }
- }
- }
- }
- if (null == address) {
- throw new NoBrokersAvailableException("No host is available.");
- }
- return address;
- }
-
- @Override
- public void removeHost(SocketAddress host, Throwable reason) {
- synchronized (hostSet) {
- if (hostSet.remove(host)) {
- logger.info("Node {} left due to : ", host, reason);
- }
- hostList = new ArrayList<SocketAddress>(hostSet);
- Collections.sort(hostList, HostComparator.INSTANCE);
- logger.info("Host list becomes : {}.", hostList);
- }
- }
-
- @Override
- public void run() {
- try {
- serverSetWatcher.watch(new ServerSetWatcher.ServerSetMonitor() {
- @Override
- public void onChange(ImmutableSet<DLSocketAddress> serviceInstances) {
- ImmutableSet<DLSocketAddress> lastValue = serverSetChange.getAndSet(serviceInstances);
- if (null == lastValue) {
- ImmutableSet<DLSocketAddress> mostRecentValue;
- do {
- mostRecentValue = serverSetChange.get();
- performServerSetChange(mostRecentValue);
- changeLatch.countDown();
- } while (!serverSetChange.compareAndSet(mostRecentValue, null));
- }
- }
- });
- } catch (Exception e) {
- logger.error("Fail to monitor server set : ", e);
- Runtime.getRuntime().exit(-1);
- }
- }
-
- protected synchronized void performServerSetChange(ImmutableSet<DLSocketAddress> serverSet) {
- Set<SocketAddress> newSet = new HashSet<SocketAddress>();
- for (DLSocketAddress serviceInstance : serverSet) {
- newSet.add(serviceInstance.getSocketAddress());
- }
-
- Set<SocketAddress> removed;
- Set<SocketAddress> added;
- synchronized (hostSet) {
- removed = Sets.difference(hostSet, newSet).immutableCopy();
- added = Sets.difference(newSet, hostSet).immutableCopy();
- for (SocketAddress node: removed) {
- if (hostSet.remove(node)) {
- logger.info("Node {} left.", node);
- }
- }
- for (SocketAddress node: added) {
- if (hostSet.add(node)) {
- logger.info("Node {} joined.", node);
- }
- }
- }
-
- for (SocketAddress addr : removed) {
- for (RoutingListener listener : listeners) {
- listener.onServerLeft(addr);
- }
- }
-
- for (SocketAddress addr : added) {
- for (RoutingListener listener : listeners) {
- listener.onServerJoin(addr);
- }
- }
-
- synchronized (hostSet) {
- hostList = new ArrayList<SocketAddress>(hostSet);
- Collections.sort(hostList, HostComparator.INSTANCE);
- logger.info("Host list becomes : {}.", hostList);
- }
-
- }
-
- static int signSafeMod(long dividend, int divisor) {
- int mod = (int) (dividend % divisor);
-
- if (mod < 0) {
- mod += divisor;
- }
-
- return mod;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetWatcher.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetWatcher.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetWatcher.java
deleted file mode 100644
index 1eccb63..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetWatcher.java
+++ /dev/null
@@ -1,71 +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.client.routing;
-
-import com.google.common.collect.ImmutableSet;
-import com.twitter.distributedlog.service.DLSocketAddress;
-
-/**
- * Watch on server set changes.
- */
-public interface ServerSetWatcher {
-
- /**
- * Exception thrown when failed to monitor serverset.
- */
- class MonitorException extends Exception {
-
- private static final long serialVersionUID = 392751505154339548L;
-
- public MonitorException(String msg) {
- super(msg);
- }
-
- public MonitorException(String msg, Throwable cause) {
- super(msg, cause);
- }
- }
-
- /**
- * An interface to an object that is interested in receiving notification whenever the host set changes.
- */
- interface ServerSetMonitor {
-
- /**
- * Called when either the available set of services changes.
- *
- * <p>It happens either when a service dies or a new INSTANCE comes on-line or
- * when an existing service advertises a status or health change.
- *
- * @param hostSet the current set of available ServiceInstances
- */
- void onChange(ImmutableSet<DLSocketAddress> hostSet);
- }
-
- /**
- * Registers a monitor to receive change notices for this server set as long as this jvm process is alive.
- *
- * <p>Blocks until the initial server set can be gathered and delivered to the monitor.
- * The monitor will be notified if the membership set or parameters of existing members have
- * changed.
- *
- * @param monitor the server set monitor to call back when the host set changes
- * @throws MonitorException if there is a problem monitoring the host set
- */
- void watch(final ServerSetMonitor monitor) throws MonitorException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/SingleHostRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/SingleHostRoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/SingleHostRoutingService.java
deleted file mode 100644
index e526868..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/SingleHostRoutingService.java
+++ /dev/null
@@ -1,128 +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.client.routing;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.Sets;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-
-/**
- * Single Host Routing Service.
- */
-public class SingleHostRoutingService implements RoutingService {
-
- public static SingleHostRoutingService of(SocketAddress address) {
- return new SingleHostRoutingService(address);
- }
-
- /**
- * Builder to build single host based routing service.
- *
- * @return builder to build single host based routing service.
- */
- public static Builder newBuilder() {
- return new Builder();
- }
-
- /**
- * Builder to build single host based routing service.
- */
- public static class Builder implements RoutingService.Builder {
-
- private SocketAddress address;
-
- private Builder() {}
-
- public Builder address(SocketAddress address) {
- this.address = address;
- return this;
- }
-
- @Override
- public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
- return this;
- }
-
- @Override
- public RoutingService build() {
- checkNotNull(address, "Host is null");
- return new SingleHostRoutingService(address);
- }
- }
-
- private SocketAddress address;
- private final CopyOnWriteArraySet<RoutingListener> listeners =
- new CopyOnWriteArraySet<RoutingListener>();
-
- SingleHostRoutingService(SocketAddress address) {
- this.address = address;
- }
-
- public void setAddress(SocketAddress address) {
- this.address = address;
- }
-
- @Override
- public Set<SocketAddress> getHosts() {
- return Sets.newHashSet(address);
- }
-
- @Override
- public void startService() {
- // no-op
- for (RoutingListener listener : listeners) {
- listener.onServerJoin(address);
- }
- }
-
- @Override
- public void stopService() {
- // no-op
- }
-
- @Override
- public RoutingService registerListener(RoutingListener listener) {
- listeners.add(listener);
- return this;
- }
-
- @Override
- public RoutingService unregisterListener(RoutingListener listener) {
- listeners.remove(listener);
- return null;
- }
-
- @Override
- public SocketAddress getHost(String key, RoutingContext rContext)
- throws NoBrokersAvailableException {
- if (rContext.isTriedHost(address)) {
- throw new NoBrokersAvailableException("No hosts is available : routing context = " + rContext);
- }
- return address;
- }
-
- @Override
- public void removeHost(SocketAddress address, Throwable reason) {
- // no-op
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TestName.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TestName.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TestName.java
deleted file mode 100644
index 8101075..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TestName.java
+++ /dev/null
@@ -1,49 +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.client.routing;
-
-import com.twitter.finagle.Addr;
-import com.twitter.finagle.Address;
-import com.twitter.finagle.Addrs;
-import com.twitter.finagle.Name;
-import java.util.List;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-/**
- * A {@link Name} implementation for testing purpose.
- */
-public class TestName implements Name {
-
- private static final Logger LOG = LoggerFactory.getLogger(TestName.class);
-
- private AbstractFunction1<Addr, BoxedUnit> callback = null;
-
- public void changes(AbstractFunction1<Addr, BoxedUnit> callback) {
- this.callback = callback;
- }
-
- public void changeAddrs(List<Address> addresses) {
- if (null != callback) {
- LOG.info("Sending a callback {}", addresses);
- callback.apply(Addrs.newBoundAddr(addresses));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TwitterServerSetWatcher.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TwitterServerSetWatcher.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TwitterServerSetWatcher.java
deleted file mode 100644
index cffb9b9..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TwitterServerSetWatcher.java
+++ /dev/null
@@ -1,83 +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.client.routing;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import com.twitter.common.net.pool.DynamicHostSet;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.thrift.Endpoint;
-import com.twitter.thrift.ServiceInstance;
-import java.net.InetSocketAddress;
-import java.util.Set;
-
-/**
- * Twitter {@link ServerSet} based watcher.
- */
-public class TwitterServerSetWatcher implements ServerSetWatcher {
-
- private final ServerSet serverSet;
- private final boolean resolvedFromName;
-
- /**
- * Construct a {@link ServerSet} based watcher.
- *
- * @param serverSet server set.
- * @param resolvedFromName whether to resolve hosts from {@link com.twitter.finagle.Name}.
- */
- public TwitterServerSetWatcher(ServerSet serverSet,
- boolean resolvedFromName) {
- this.serverSet = serverSet;
- this.resolvedFromName = resolvedFromName;
- }
-
- /**
- * Registers a monitor to receive change notices for this server set as long as this jvm process is alive.
- *
- * <p>Blocks until the initial server set can be gathered and delivered to the monitor.
- * The monitor will be notified if the membership set or parameters of existing members have
- * changed.
- *
- * @param monitor the server set monitor to call back when the host set changes
- * @throws MonitorException if there is a problem monitoring the host set
- */
- public void watch(final ServerSetMonitor monitor)
- throws MonitorException {
- try {
- serverSet.watch(new DynamicHostSet.HostChangeMonitor<ServiceInstance>() {
- @Override
- public void onChange(ImmutableSet<ServiceInstance> serviceInstances) {
- Set<DLSocketAddress> dlServers = Sets.newHashSet();
- for (ServiceInstance serviceInstance : serviceInstances) {
- Endpoint endpoint = serviceInstance.getAdditionalEndpoints().get("thrift");
- InetSocketAddress inetAddr =
- new InetSocketAddress(endpoint.getHost(), endpoint.getPort());
- int shardId = resolvedFromName ? -1 : serviceInstance.getShard();
- DLSocketAddress address = new DLSocketAddress(shardId, inetAddr);
- dlServers.add(address);
- }
- monitor.onChange(ImmutableSet.copyOf(dlServers));
- }
- });
- } catch (DynamicHostSet.MonitorException me) {
- throw new MonitorException("Failed to monitor server set : ", me);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/package-info.java
deleted file mode 100644
index a282b42..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Routing Mechanisms to route the traffic to the owner of streams.
- */
-package com.twitter.distributedlog.client.routing;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/DLZkServerSet.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/DLZkServerSet.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/DLZkServerSet.java
deleted file mode 100644
index 4ca3aa6..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/DLZkServerSet.java
+++ /dev/null
@@ -1,91 +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.client.serverset;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.net.HostAndPort;
-import com.twitter.common.quantity.Amount;
-import com.twitter.common.quantity.Time;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.common.zookeeper.ServerSets;
-import com.twitter.common.zookeeper.ZooKeeperClient;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import org.apache.commons.lang.StringUtils;
-import org.apache.zookeeper.ZooDefs;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A wrapper over zookeeper client and its server set.
- */
-public class DLZkServerSet {
-
- private static final Logger logger = LoggerFactory.getLogger(DLZkServerSet.class);
-
- static final String ZNODE_WRITE_PROXY = ".write_proxy";
-
- private static String getZKServersFromDLUri(URI uri) {
- return uri.getAuthority().replace(";", ",");
- }
-
- private static Iterable<InetSocketAddress> getZkAddresses(URI uri) {
- String zkServers = getZKServersFromDLUri(uri);
- String[] zkServerList = StringUtils.split(zkServers, ',');
- ImmutableList.Builder<InetSocketAddress> builder = ImmutableList.builder();
- for (String zkServer : zkServerList) {
- HostAndPort hostAndPort = HostAndPort.fromString(zkServer).withDefaultPort(2181);
- builder.add(InetSocketAddress.createUnresolved(
- hostAndPort.getHostText(),
- hostAndPort.getPort()));
- }
- return builder.build();
- }
-
- public static DLZkServerSet of(URI uri,
- int zkSessionTimeoutMs) {
- // Create zookeeper and server set
- String zkPath = uri.getPath() + "/" + ZNODE_WRITE_PROXY;
- Iterable<InetSocketAddress> zkAddresses = getZkAddresses(uri);
- ZooKeeperClient zkClient =
- new ZooKeeperClient(Amount.of(zkSessionTimeoutMs, Time.MILLISECONDS), zkAddresses);
- ServerSet serverSet = ServerSets.create(zkClient, ZooDefs.Ids.OPEN_ACL_UNSAFE, zkPath);
- return new DLZkServerSet(zkClient, serverSet);
- }
-
- private final ZooKeeperClient zkClient;
- private final ServerSet zkServerSet;
-
- public DLZkServerSet(ZooKeeperClient zkClient,
- ServerSet zkServerSet) {
- this.zkClient = zkClient;
- this.zkServerSet = zkServerSet;
- }
-
- public ZooKeeperClient getZkClient() {
- return zkClient;
- }
-
- public ServerSet getServerSet() {
- return zkServerSet;
- }
-
- public void close() {
- zkClient.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/package-info.java
deleted file mode 100644
index 49166ec..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Utils related to server set.
- */
-package com.twitter.distributedlog.client.serverset;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
deleted file mode 100644
index 489fc00..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
+++ /dev/null
@@ -1,119 +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.client.speculative;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Default implementation of {@link SpeculativeRequestExecutionPolicy}.
- */
-public class DefaultSpeculativeRequestExecutionPolicy implements SpeculativeRequestExecutionPolicy {
-
- private static final Logger LOG = LoggerFactory.getLogger(DefaultSpeculativeRequestExecutionPolicy.class);
- final int firstSpeculativeRequestTimeout;
- final int maxSpeculativeRequestTimeout;
- final float backoffMultiplier;
- int nextSpeculativeRequestTimeout;
-
- public DefaultSpeculativeRequestExecutionPolicy(int firstSpeculativeRequestTimeout,
- int maxSpeculativeRequestTimeout,
- float backoffMultiplier) {
- this.firstSpeculativeRequestTimeout = firstSpeculativeRequestTimeout;
- this.maxSpeculativeRequestTimeout = maxSpeculativeRequestTimeout;
- this.backoffMultiplier = backoffMultiplier;
- this.nextSpeculativeRequestTimeout = firstSpeculativeRequestTimeout;
-
- if (backoffMultiplier <= 0) {
- throw new IllegalArgumentException("Invalid value provided for backoffMultiplier");
- }
-
- // Prevent potential over flow
- if (Math.round((double) maxSpeculativeRequestTimeout * (double) backoffMultiplier) > Integer.MAX_VALUE) {
- throw new IllegalArgumentException("Invalid values for maxSpeculativeRequestTimeout and backoffMultiplier");
- }
- }
-
- @VisibleForTesting
- int getNextSpeculativeRequestTimeout() {
- return nextSpeculativeRequestTimeout;
- }
-
- /**
- * Initialize the speculative request execution policy.
- *
- * @param scheduler The scheduler service to issue the speculative request
- * @param requestExecutor The executor is used to issue the actual speculative requests
- */
- @Override
- public void initiateSpeculativeRequest(final ScheduledExecutorService scheduler,
- final SpeculativeRequestExecutor requestExecutor) {
- issueSpeculativeRequest(scheduler, requestExecutor);
- }
-
- private void issueSpeculativeRequest(final ScheduledExecutorService scheduler,
- final SpeculativeRequestExecutor requestExecutor) {
- Future<Boolean> issueNextRequest = requestExecutor.issueSpeculativeRequest();
- issueNextRequest.addEventListener(new FutureEventListener<Boolean>() {
- // we want this handler to run immediately after we push the big red button!
- @Override
- public void onSuccess(Boolean issueNextRequest) {
- if (issueNextRequest) {
- scheduleSpeculativeRequest(scheduler, requestExecutor, nextSpeculativeRequestTimeout);
- nextSpeculativeRequestTimeout = Math.min(maxSpeculativeRequestTimeout,
- (int) (nextSpeculativeRequestTimeout * backoffMultiplier));
- } else {
- if (LOG.isTraceEnabled()) {
- LOG.trace("Stopped issuing speculative requests for {}, "
- + "speculativeReadTimeout = {}", requestExecutor, nextSpeculativeRequestTimeout);
- }
- }
- }
-
- @Override
- public void onFailure(Throwable thrown) {
- LOG.warn("Failed to issue speculative request for {}, speculativeReadTimeout = {} : ",
- new Object[] { requestExecutor, nextSpeculativeRequestTimeout, thrown });
- }
- });
- }
-
- private void scheduleSpeculativeRequest(final ScheduledExecutorService scheduler,
- final SpeculativeRequestExecutor requestExecutor,
- final int speculativeRequestTimeout) {
- try {
- scheduler.schedule(new Runnable() {
- @Override
- public void run() {
- issueSpeculativeRequest(scheduler, requestExecutor);
- }
- }, speculativeRequestTimeout, TimeUnit.MILLISECONDS);
- } catch (RejectedExecutionException re) {
- if (!scheduler.isShutdown()) {
- LOG.warn("Failed to schedule speculative request for {}, speculativeReadTimeout = {} : ",
- new Object[]{requestExecutor, speculativeRequestTimeout, re});
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
deleted file mode 100644
index 14615e9..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
+++ /dev/null
@@ -1,34 +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.client.speculative;
-
-import java.util.concurrent.ScheduledExecutorService;
-
-/**
- * Speculative request execution policy.
- */
-public interface SpeculativeRequestExecutionPolicy {
- /**
- * Initialize the speculative request execution policy and initiate requests.
- *
- * @param scheduler The scheduler service to issue the speculative request
- * @param requestExecutor The executor is used to issue the actual speculative requests
- */
- void initiateSpeculativeRequest(ScheduledExecutorService scheduler,
- SpeculativeRequestExecutor requestExecutor);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutor.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutor.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutor.java
deleted file mode 100644
index de1b0dd..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutor.java
+++ /dev/null
@@ -1,33 +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.client.speculative;
-
-import com.twitter.util.Future;
-
-/**
- * Executor to execute speculative requests.
- */
-public interface SpeculativeRequestExecutor {
-
- /**
- * Issues a speculative request and indicates if more speculative requests should be issued.
- *
- * @return whether more speculative requests should be issued.
- */
- Future<Boolean> issueSpeculativeRequest();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/package-info.java
deleted file mode 100644
index b299266..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Speculative Mechanism.
- */
-package com.twitter.distributedlog.client.speculative;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStats.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStats.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStats.java
deleted file mode 100644
index f361892..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStats.java
+++ /dev/null
@@ -1,108 +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.client.stats;
-
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Client Stats.
- */
-public class ClientStats {
-
- // Region Resolver
- private final RegionResolver regionResolver;
-
- // Stats
- private final StatsReceiver statsReceiver;
- private final ClientStatsLogger clientStatsLogger;
- private final boolean enableRegionStats;
- private final ConcurrentMap<String, ClientStatsLogger> regionClientStatsLoggers;
- private final ConcurrentMap<String, OpStats> opStatsMap;
-
- public ClientStats(StatsReceiver statsReceiver,
- boolean enableRegionStats,
- RegionResolver regionResolver) {
- this.statsReceiver = statsReceiver;
- this.clientStatsLogger = new ClientStatsLogger(statsReceiver);
- this.enableRegionStats = enableRegionStats;
- this.regionClientStatsLoggers = new ConcurrentHashMap<String, ClientStatsLogger>();
- this.regionResolver = regionResolver;
- this.opStatsMap = new ConcurrentHashMap<String, OpStats>();
- }
-
- public OpStats getOpStats(String op) {
- OpStats opStats = opStatsMap.get(op);
- if (null != opStats) {
- return opStats;
- }
- OpStats newStats = new OpStats(statsReceiver.scope(op),
- enableRegionStats, regionResolver);
- OpStats oldStats = opStatsMap.putIfAbsent(op, newStats);
- if (null == oldStats) {
- return newStats;
- } else {
- return oldStats;
- }
- }
-
- private ClientStatsLogger getRegionClientStatsLogger(SocketAddress address) {
- String region = regionResolver.resolveRegion(address);
- return getRegionClientStatsLogger(region);
- }
-
- private ClientStatsLogger getRegionClientStatsLogger(String region) {
- ClientStatsLogger statsLogger = regionClientStatsLoggers.get(region);
- if (null == statsLogger) {
- ClientStatsLogger newStatsLogger = new ClientStatsLogger(statsReceiver.scope(region));
- ClientStatsLogger oldStatsLogger = regionClientStatsLoggers.putIfAbsent(region, newStatsLogger);
- if (null == oldStatsLogger) {
- statsLogger = newStatsLogger;
- } else {
- statsLogger = oldStatsLogger;
- }
- }
- return statsLogger;
- }
-
- public StatsReceiver getFinagleStatsReceiver(SocketAddress addr) {
- if (enableRegionStats && null != addr) {
- return getRegionClientStatsLogger(addr).getStatsReceiver();
- } else {
- return clientStatsLogger.getStatsReceiver();
- }
- }
-
- public void completeProxyRequest(SocketAddress addr, StatusCode code, long startTimeNanos) {
- clientStatsLogger.completeProxyRequest(code, startTimeNanos);
- if (enableRegionStats && null != addr) {
- getRegionClientStatsLogger(addr).completeProxyRequest(code, startTimeNanos);
- }
- }
-
- public void failProxyRequest(SocketAddress addr, Throwable cause, long startTimeNanos) {
- clientStatsLogger.failProxyRequest(cause, startTimeNanos);
- if (enableRegionStats && null != addr) {
- getRegionClientStatsLogger(addr).failProxyRequest(cause, startTimeNanos);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStatsLogger.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStatsLogger.java
deleted file mode 100644
index 0df64cc..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStatsLogger.java
+++ /dev/null
@@ -1,91 +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.client.stats;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.finagle.stats.Counter;
-import com.twitter.finagle.stats.Stat;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Stats Logger to collect client stats.
- */
-public class ClientStatsLogger {
-
- // Stats
- private final StatsReceiver statsReceiver;
- private final StatsReceiver responseStatsReceiver;
- private final ConcurrentMap<StatusCode, Counter> responseStats =
- new ConcurrentHashMap<StatusCode, Counter>();
- private final StatsReceiver exceptionStatsReceiver;
- private final ConcurrentMap<Class<?>, Counter> exceptionStats =
- new ConcurrentHashMap<Class<?>, Counter>();
-
- private final Stat proxySuccessLatencyStat;
- private final Stat proxyFailureLatencyStat;
-
- public ClientStatsLogger(StatsReceiver statsReceiver) {
- this.statsReceiver = statsReceiver;
- responseStatsReceiver = statsReceiver.scope("responses");
- exceptionStatsReceiver = statsReceiver.scope("exceptions");
- StatsReceiver proxyLatencyStatReceiver = statsReceiver.scope("proxy_request_latency");
- proxySuccessLatencyStat = proxyLatencyStatReceiver.stat0("success");
- proxyFailureLatencyStat = proxyLatencyStatReceiver.stat0("failure");
- }
-
- public StatsReceiver getStatsReceiver() {
- return statsReceiver;
- }
-
- private Counter getResponseCounter(StatusCode code) {
- Counter counter = responseStats.get(code);
- if (null == counter) {
- Counter newCounter = responseStatsReceiver.counter0(code.name());
- Counter oldCounter = responseStats.putIfAbsent(code, newCounter);
- counter = null != oldCounter ? oldCounter : newCounter;
- }
- return counter;
- }
-
- private Counter getExceptionCounter(Class<?> cls) {
- Counter counter = exceptionStats.get(cls);
- if (null == counter) {
- Counter newCounter = exceptionStatsReceiver.counter0(cls.getName());
- Counter oldCounter = exceptionStats.putIfAbsent(cls, newCounter);
- counter = null != oldCounter ? oldCounter : newCounter;
- }
- return counter;
- }
-
- public void completeProxyRequest(StatusCode code, long startTimeNanos) {
- getResponseCounter(code).incr();
- proxySuccessLatencyStat.add(elapsedMicroSec(startTimeNanos));
- }
-
- public void failProxyRequest(Throwable cause, long startTimeNanos) {
- getExceptionCounter(cause.getClass()).incr();
- proxyFailureLatencyStat.add(elapsedMicroSec(startTimeNanos));
- }
-
- static long elapsedMicroSec(long startNanoTime) {
- return TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startNanoTime);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStats.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStats.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStats.java
deleted file mode 100644
index 26708f3..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStats.java
+++ /dev/null
@@ -1,82 +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.client.stats;
-
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Op Stats.
- */
-public class OpStats {
-
- // Region Resolver
- private final RegionResolver regionResolver;
-
- // Stats
- private final StatsReceiver statsReceiver;
- private final OpStatsLogger opStatsLogger;
- private final boolean enableRegionStats;
- private final ConcurrentMap<String, OpStatsLogger> regionOpStatsLoggers;
-
- public OpStats(StatsReceiver statsReceiver,
- boolean enableRegionStats,
- RegionResolver regionResolver) {
- this.statsReceiver = statsReceiver;
- this.opStatsLogger = new OpStatsLogger(statsReceiver);
- this.enableRegionStats = enableRegionStats;
- this.regionOpStatsLoggers = new ConcurrentHashMap<String, OpStatsLogger>();
- this.regionResolver = regionResolver;
- }
-
- private OpStatsLogger getRegionOpStatsLogger(SocketAddress address) {
- String region = regionResolver.resolveRegion(address);
- return getRegionOpStatsLogger(region);
- }
-
- private OpStatsLogger getRegionOpStatsLogger(String region) {
- OpStatsLogger statsLogger = regionOpStatsLoggers.get(region);
- if (null == statsLogger) {
- OpStatsLogger newStatsLogger = new OpStatsLogger(statsReceiver.scope(region));
- OpStatsLogger oldStatsLogger = regionOpStatsLoggers.putIfAbsent(region, newStatsLogger);
- if (null == oldStatsLogger) {
- statsLogger = newStatsLogger;
- } else {
- statsLogger = oldStatsLogger;
- }
- }
- return statsLogger;
- }
-
- public void completeRequest(SocketAddress addr, long micros, int numTries) {
- opStatsLogger.completeRequest(micros, numTries);
- if (enableRegionStats && null != addr) {
- getRegionOpStatsLogger(addr).completeRequest(micros, numTries);
- }
- }
-
- public void failRequest(SocketAddress addr, long micros, int numTries) {
- opStatsLogger.failRequest(micros, numTries);
- if (enableRegionStats && null != addr) {
- getRegionOpStatsLogger(addr).failRequest(micros, numTries);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStatsLogger.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStatsLogger.java
deleted file mode 100644
index 27adda7..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStatsLogger.java
+++ /dev/null
@@ -1,50 +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.client.stats;
-
-import com.twitter.finagle.stats.Stat;
-import com.twitter.finagle.stats.StatsReceiver;
-
-/**
- * Stats Logger per operation type.
- */
-public class OpStatsLogger {
-
- private final Stat successLatencyStat;
- private final Stat failureLatencyStat;
- private final Stat redirectStat;
-
- public OpStatsLogger(StatsReceiver statsReceiver) {
- StatsReceiver latencyStatReceiver = statsReceiver.scope("latency");
- successLatencyStat = latencyStatReceiver.stat0("success");
- failureLatencyStat = latencyStatReceiver.stat0("failure");
- StatsReceiver redirectStatReceiver = statsReceiver.scope("redirects");
- redirectStat = redirectStatReceiver.stat0("times");
- }
-
- public void completeRequest(long micros, int numTries) {
- successLatencyStat.add(micros);
- redirectStat.add(numTries);
- }
-
- public void failRequest(long micros, int numTries) {
- failureLatencyStat.add(micros);
- redirectStat.add(numTries);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OwnershipStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OwnershipStatsLogger.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OwnershipStatsLogger.java
deleted file mode 100644
index 7d5a9c9..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OwnershipStatsLogger.java
+++ /dev/null
@@ -1,115 +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.client.stats;
-
-import com.twitter.finagle.stats.Counter;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Stats Logger for ownerships.
- */
-public class OwnershipStatsLogger {
-
- /**
- * Ownership related stats.
- */
- public static class OwnershipStat {
- private final Counter hits;
- private final Counter misses;
- private final Counter removes;
- private final Counter redirects;
- private final Counter adds;
-
- OwnershipStat(StatsReceiver ownershipStats) {
- hits = ownershipStats.counter0("hits");
- misses = ownershipStats.counter0("misses");
- adds = ownershipStats.counter0("adds");
- removes = ownershipStats.counter0("removes");
- redirects = ownershipStats.counter0("redirects");
- }
-
- public void onHit() {
- hits.incr();
- }
-
- public void onMiss() {
- misses.incr();
- }
-
- public void onAdd() {
- adds.incr();
- }
-
- public void onRemove() {
- removes.incr();
- }
-
- public void onRedirect() {
- redirects.incr();
- }
-
- }
-
- private final OwnershipStat ownershipStat;
- private final StatsReceiver ownershipStatsReceiver;
- private final ConcurrentMap<String, OwnershipStat> ownershipStats =
- new ConcurrentHashMap<String, OwnershipStat>();
-
- public OwnershipStatsLogger(StatsReceiver statsReceiver,
- StatsReceiver streamStatsReceiver) {
- this.ownershipStat = new OwnershipStat(statsReceiver.scope("ownership"));
- this.ownershipStatsReceiver = streamStatsReceiver.scope("perstream_ownership");
- }
-
- private OwnershipStat getOwnershipStat(String stream) {
- OwnershipStat stat = ownershipStats.get(stream);
- if (null == stat) {
- OwnershipStat newStat = new OwnershipStat(ownershipStatsReceiver.scope(stream));
- OwnershipStat oldStat = ownershipStats.putIfAbsent(stream, newStat);
- stat = null != oldStat ? oldStat : newStat;
- }
- return stat;
- }
-
- public void onMiss(String stream) {
- ownershipStat.onMiss();
- getOwnershipStat(stream).onMiss();
- }
-
- public void onHit(String stream) {
- ownershipStat.onHit();
- getOwnershipStat(stream).onHit();
- }
-
- public void onRedirect(String stream) {
- ownershipStat.onRedirect();
- getOwnershipStat(stream).onRedirect();
- }
-
- public void onRemove(String stream) {
- ownershipStat.onRemove();
- getOwnershipStat(stream).onRemove();
- }
-
- public void onAdd(String stream) {
- ownershipStat.onAdd();
- getOwnershipStat(stream).onAdd();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/package-info.java
deleted file mode 100644
index 91d4f39..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Client side stats utils.
- */
-package com.twitter.distributedlog.client.stats;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DLSocketAddress.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DLSocketAddress.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DLSocketAddress.java
deleted file mode 100644
index 30891c0..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DLSocketAddress.java
+++ /dev/null
@@ -1,161 +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.service;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-
-/**
- * Socket Address identifier for a DL proxy.
- */
-public class DLSocketAddress {
-
- private static final int VERSION = 1;
-
- private static final String COLON = ":";
- private static final String SEP = ";";
-
- private final int shard;
- private final InetSocketAddress socketAddress;
-
- public DLSocketAddress(int shard, InetSocketAddress socketAddress) {
- this.shard = shard;
- this.socketAddress = socketAddress;
- }
-
- /**
- * Shard id for dl write proxy.
- *
- * @return shard id for dl write proxy.
- */
- public int getShard() {
- return shard;
- }
-
- /**
- * Socket address for dl write proxy.
- *
- * @return socket address for dl write proxy
- */
- public InetSocketAddress getSocketAddress() {
- return socketAddress;
- }
-
- /**
- * Serialize the write proxy identifier to string.
- *
- * @return serialized write proxy identifier.
- */
- public String serialize() {
- return toLockId(socketAddress, shard);
- }
-
- @Override
- public int hashCode() {
- return socketAddress.hashCode();
- }
-
- @Override
- public boolean equals(Object obj) {
- if (!(obj instanceof DLSocketAddress)) {
- return false;
- }
- DLSocketAddress other = (DLSocketAddress) obj;
- return shard == other.shard && socketAddress.equals(other.socketAddress);
- }
-
- @Override
- public String toString() {
- return toLockId(socketAddress, shard);
- }
-
- /**
- * Deserialize proxy address from a string representation.
- *
- * @param lockId
- * string representation of the proxy address.
- * @return proxy address.
- * @throws IOException
- */
- public static DLSocketAddress deserialize(String lockId) throws IOException {
- String parts[] = lockId.split(SEP);
- if (3 != parts.length) {
- throw new IOException("Invalid dl socket address " + lockId);
- }
- int version;
- try {
- version = Integer.parseInt(parts[0]);
- } catch (NumberFormatException nfe) {
- throw new IOException("Invalid version found in " + lockId, nfe);
- }
- if (VERSION != version) {
- throw new IOException("Invalid version " + version + " found in " + lockId + ", expected " + VERSION);
- }
- int shardId;
- try {
- shardId = Integer.parseInt(parts[1]);
- } catch (NumberFormatException nfe) {
- throw new IOException("Invalid shard id found in " + lockId, nfe);
- }
- InetSocketAddress address = parseSocketAddress(parts[2]);
- return new DLSocketAddress(shardId, address);
- }
-
- /**
- * Parse the inet socket address from the string representation.
- *
- * @param addr
- * string representation
- * @return inet socket address
- */
- public static InetSocketAddress parseSocketAddress(String addr) {
- String[] parts = addr.split(COLON);
- checkArgument(parts.length == 2);
- String hostname = parts[0];
- int port = Integer.parseInt(parts[1]);
- return new InetSocketAddress(hostname, port);
- }
-
- public static InetSocketAddress getSocketAddress(int port) throws UnknownHostException {
- return new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), port);
- }
-
- /**
- * Convert inet socket address to the string representation.
- *
- * @param address
- * inet socket address.
- * @return string representation of inet socket address.
- */
- public static String toString(InetSocketAddress address) {
- StringBuilder sb = new StringBuilder();
- sb.append(address.getHostName()).append(COLON).append(address.getPort());
- return sb.toString();
- }
-
- public static String toLockId(InetSocketAddress address, int shard) {
- StringBuilder sb = new StringBuilder();
- sb.append(VERSION).append(SEP).append(shard).append(SEP).append(toString(address));
- return sb.toString();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClient.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClient.java
deleted file mode 100644
index a2c5150..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClient.java
+++ /dev/null
@@ -1,108 +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.service;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordSetBuffer;
-import com.twitter.util.Future;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-/**
- * Interface for distributedlog client.
- */
-public interface DistributedLogClient {
- /**
- * Write <i>data</i> to a given <i>stream</i>.
- *
- * @param stream
- * Stream Name.
- * @param data
- * Data to write.
- * @return a future representing a sequence id returned for this write.
- */
- Future<DLSN> write(String stream, ByteBuffer data);
-
- /**
- * Write record set to a given <i>stream</i>.
- *
- * <p>The record set is built from {@link com.twitter.distributedlog.LogRecordSet.Writer}
- *
- * @param stream stream to write to
- * @param recordSet record set
- */
- Future<DLSN> writeRecordSet(String stream, LogRecordSetBuffer recordSet);
-
- /**
- * Write <i>data</i> in bulk to a given <i>stream</i>.
- *
- * <p>Return a list of Future dlsns, one for each submitted buffer. In the event of a partial
- * failure--ex. some specific buffer write fails, all subsequent writes
- * will also fail.
- *
- * @param stream
- * Stream Name.
- * @param data
- * Data to write.
- * @return a list of futures, one for each submitted buffer.
- */
- List<Future<DLSN>> writeBulk(String stream, List<ByteBuffer> data);
-
- /**
- * Truncate the stream to a given <i>dlsn</i>.
- *
- * @param stream
- * Stream Name.
- * @param dlsn
- * DLSN to truncate until.
- * @return a future representing the truncation.
- */
- Future<Boolean> truncate(String stream, DLSN dlsn);
-
- /**
- * Release the ownership of a stream <i>stream</i>.
- *
- * @param stream
- * Stream Name to release.
- * @return a future representing the release operation.
- */
- Future<Void> release(String stream);
-
- /**
- * Delete a given stream <i>stream</i>.
- *
- * @param stream
- * Stream Name to delete.
- * @return a future representing the delete operation.
- */
- Future<Void> delete(String stream);
-
- /**
- * Create a stream with name <i>stream</i>.
- *
- * @param stream
- * Stream Name to create.
- * @return a future representing the create operation.
- */
- Future<Void> create(String stream);
-
- /**
- * Close the client.
- */
- void close();
-}
[51/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
DL-4: Repackage the source under apache namespace
Project: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/commit/b44820b5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/tree/b44820b5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/diff/b44820b5
Branch: refs/heads/master
Commit: b44820b50eea9ef2447ec7e136e182ba39e2c269
Parents: 7fab246
Author: Sijie Guo <si...@twitter.com>
Authored: Wed Jan 4 16:44:15 2017 -0800
Committer: Sijie Guo <si...@twitter.com>
Committed: Wed Jan 4 16:46:19 2017 -0800
----------------------------------------------------------------------
distributedlog-benchmark/conf/log4j.properties | 8 +-
.../distributedlog/benchmark/Benchmarker.java | 468 ---
.../benchmark/DLWriterWorker.java | 245 --
.../distributedlog/benchmark/ReaderWorker.java | 468 ---
.../twitter/distributedlog/benchmark/Utils.java | 57 -
.../distributedlog/benchmark/Worker.java | 26 -
.../distributedlog/benchmark/WriterWorker.java | 387 --
.../distributedlog/benchmark/package-info.java | 21 -
.../stream/AbstractReaderBenchmark.java | 70 -
.../benchmark/stream/AsyncReaderBenchmark.java | 158 -
.../benchmark/stream/LedgerBatchReader.java | 82 -
.../benchmark/stream/LedgerReadBenchmark.java | 151 -
.../benchmark/stream/LedgerStreamReader.java | 131 -
.../benchmark/stream/ReadMode.java | 28 -
.../benchmark/stream/StreamBenchmark.java | 138 -
.../benchmark/stream/SyncReaderBenchmark.java | 164 -
.../benchmark/stream/package-info.java | 21 -
.../benchmark/utils/ShiftableRateLimiter.java | 71 -
.../benchmark/utils/package-info.java | 21 -
.../distributedlog/benchmark/Benchmarker.java | 468 +++
.../benchmark/DLWriterWorker.java | 245 ++
.../distributedlog/benchmark/ReaderWorker.java | 468 +++
.../apache/distributedlog/benchmark/Utils.java | 57 +
.../apache/distributedlog/benchmark/Worker.java | 26 +
.../distributedlog/benchmark/WriterWorker.java | 387 ++
.../distributedlog/benchmark/package-info.java | 21 +
.../stream/AbstractReaderBenchmark.java | 70 +
.../benchmark/stream/AsyncReaderBenchmark.java | 158 +
.../benchmark/stream/LedgerBatchReader.java | 82 +
.../benchmark/stream/LedgerReadBenchmark.java | 151 +
.../benchmark/stream/LedgerStreamReader.java | 131 +
.../benchmark/stream/ReadMode.java | 28 +
.../benchmark/stream/StreamBenchmark.java | 138 +
.../benchmark/stream/SyncReaderBenchmark.java | 164 +
.../benchmark/stream/package-info.java | 21 +
.../benchmark/utils/ShiftableRateLimiter.java | 71 +
.../benchmark/utils/package-info.java | 21 +
.../src/main/resources/findbugsExclude.xml | 2 +-
.../src/main/thrift/loadtest.thrift | 2 +-
distributedlog-client/pom.xml | 2 +-
.../distributedlog/client/ClientConfig.java | 187 -
.../client/DistributedLogClientImpl.java | 1200 ------
.../client/DistributedLogMultiStreamWriter.java | 486 ---
.../client/monitor/MonitorServiceClient.java | 68 -
.../client/monitor/package-info.java | 21 -
.../client/ownership/OwnershipCache.java | 235 --
.../client/ownership/package-info.java | 21 -
.../distributedlog/client/package-info.java | 21 -
.../client/proxy/ClusterClient.java | 51 -
.../client/proxy/HostProvider.java | 35 -
.../client/proxy/ProxyClient.java | 165 -
.../client/proxy/ProxyClientManager.java | 362 --
.../client/proxy/ProxyListener.java | 50 -
.../client/proxy/package-info.java | 21 -
.../client/resolver/DefaultRegionResolver.java | 85 -
.../client/resolver/RegionResolver.java | 43 -
.../client/resolver/package-info.java | 21 -
.../routing/ConsistentHashRoutingService.java | 500 ---
.../client/routing/NameServerSet.java | 263 --
.../client/routing/RegionsRoutingService.java | 192 -
.../client/routing/RoutingService.java | 206 -
.../client/routing/RoutingServiceProvider.java | 39 -
.../client/routing/RoutingUtils.java | 88 -
.../client/routing/ServerSetRoutingService.java | 274 --
.../client/routing/ServerSetWatcher.java | 71 -
.../routing/SingleHostRoutingService.java | 128 -
.../distributedlog/client/routing/TestName.java | 49 -
.../client/routing/TwitterServerSetWatcher.java | 83 -
.../client/routing/package-info.java | 21 -
.../client/serverset/DLZkServerSet.java | 91 -
.../client/serverset/package-info.java | 21 -
...efaultSpeculativeRequestExecutionPolicy.java | 119 -
.../SpeculativeRequestExecutionPolicy.java | 34 -
.../speculative/SpeculativeRequestExecutor.java | 33 -
.../client/speculative/package-info.java | 21 -
.../client/stats/ClientStats.java | 108 -
.../client/stats/ClientStatsLogger.java | 91 -
.../distributedlog/client/stats/OpStats.java | 82 -
.../client/stats/OpStatsLogger.java | 50 -
.../client/stats/OwnershipStatsLogger.java | 115 -
.../client/stats/package-info.java | 21 -
.../distributedlog/service/DLSocketAddress.java | 161 -
.../service/DistributedLogClient.java | 108 -
.../service/DistributedLogClientBuilder.java | 608 ---
.../distributedlog/service/package-info.java | 21 -
.../distributedlog/client/ClientConfig.java | 187 +
.../client/DistributedLogClientImpl.java | 1200 ++++++
.../client/DistributedLogMultiStreamWriter.java | 486 +++
.../client/monitor/MonitorServiceClient.java | 68 +
.../client/monitor/package-info.java | 21 +
.../client/ownership/OwnershipCache.java | 235 ++
.../client/ownership/package-info.java | 21 +
.../distributedlog/client/package-info.java | 21 +
.../client/proxy/ClusterClient.java | 51 +
.../client/proxy/HostProvider.java | 35 +
.../client/proxy/ProxyClient.java | 165 +
.../client/proxy/ProxyClientManager.java | 362 ++
.../client/proxy/ProxyListener.java | 50 +
.../client/proxy/package-info.java | 21 +
.../client/resolver/DefaultRegionResolver.java | 85 +
.../client/resolver/RegionResolver.java | 43 +
.../client/resolver/package-info.java | 21 +
.../routing/ConsistentHashRoutingService.java | 500 +++
.../client/routing/NameServerSet.java | 263 ++
.../client/routing/RegionsRoutingService.java | 192 +
.../client/routing/RoutingService.java | 206 +
.../client/routing/RoutingServiceProvider.java | 39 +
.../client/routing/RoutingUtils.java | 88 +
.../client/routing/ServerSetRoutingService.java | 274 ++
.../client/routing/ServerSetWatcher.java | 71 +
.../routing/SingleHostRoutingService.java | 128 +
.../distributedlog/client/routing/TestName.java | 49 +
.../client/routing/TwitterServerSetWatcher.java | 83 +
.../client/routing/package-info.java | 21 +
.../client/serverset/DLZkServerSet.java | 91 +
.../client/serverset/package-info.java | 21 +
...efaultSpeculativeRequestExecutionPolicy.java | 119 +
.../SpeculativeRequestExecutionPolicy.java | 34 +
.../speculative/SpeculativeRequestExecutor.java | 33 +
.../client/speculative/package-info.java | 21 +
.../client/stats/ClientStats.java | 108 +
.../client/stats/ClientStatsLogger.java | 91 +
.../distributedlog/client/stats/OpStats.java | 82 +
.../client/stats/OpStatsLogger.java | 50 +
.../client/stats/OwnershipStatsLogger.java | 115 +
.../client/stats/package-info.java | 21 +
.../distributedlog/service/DLSocketAddress.java | 161 +
.../service/DistributedLogClient.java | 108 +
.../service/DistributedLogClientBuilder.java | 608 +++
.../distributedlog/service/package-info.java | 21 +
.../src/main/resources/findbugsExclude.xml | 2 +-
.../TestDistributedLogMultiStreamWriter.java | 383 --
.../client/ownership/TestOwnershipCache.java | 207 -
.../proxy/MockDistributedLogServices.java | 144 -
.../client/proxy/MockProxyClientBuilder.java | 49 -
.../client/proxy/MockThriftClient.java | 32 -
.../client/proxy/TestProxyClientManager.java | 368 --
.../TestConsistentHashRoutingService.java | 417 ---
.../client/routing/TestInetNameResolution.java | 73 -
.../routing/TestRegionsRoutingService.java | 133 -
.../client/routing/TestRoutingService.java | 146 -
...efaultSpeculativeRequestExecutionPolicy.java | 105 -
.../TestDistributedLogClientBuilder.java | 49 -
.../TestDistributedLogMultiStreamWriter.java | 383 ++
.../client/ownership/TestOwnershipCache.java | 207 +
.../proxy/MockDistributedLogServices.java | 144 +
.../client/proxy/MockProxyClientBuilder.java | 49 +
.../client/proxy/MockThriftClient.java | 32 +
.../client/proxy/TestProxyClientManager.java | 368 ++
.../TestConsistentHashRoutingService.java | 417 +++
.../client/routing/TestInetNameResolution.java | 73 +
.../routing/TestRegionsRoutingService.java | 133 +
.../client/routing/TestRoutingService.java | 146 +
...efaultSpeculativeRequestExecutionPolicy.java | 105 +
.../TestDistributedLogClientBuilder.java | 49 +
distributedlog-core/conf/log4j.properties | 8 +-
.../conf/zookeeper.conf.dynamic.template | 2 +-
distributedlog-core/pom.xml | 2 +-
.../distributedlog/AppendOnlyStreamReader.java | 198 -
.../distributedlog/AppendOnlyStreamWriter.java | 107 -
.../twitter/distributedlog/AsyncLogReader.java | 69 -
.../twitter/distributedlog/AsyncLogWriter.java | 70 -
.../distributedlog/AsyncNotification.java | 32 -
.../distributedlog/BKAbstractLogWriter.java | 555 ---
.../distributedlog/BKAsyncLogReader.java | 751 ----
.../distributedlog/BKAsyncLogWriter.java | 559 ---
.../distributedlog/BKDistributedLogManager.java | 1106 ------
.../BKDistributedLogNamespace.java | 320 --
.../twitter/distributedlog/BKLogHandler.java | 715 ----
.../distributedlog/BKLogReadHandler.java | 431 ---
.../distributedlog/BKLogSegmentWriter.java | 1348 -------
.../distributedlog/BKLogWriteHandler.java | 1325 -------
.../twitter/distributedlog/BKSyncLogReader.java | 276 --
.../twitter/distributedlog/BKSyncLogWriter.java | 113 -
.../distributedlog/BKTransmitPacket.java | 90 -
.../distributedlog/BookKeeperClient.java | 289 --
.../distributedlog/BookKeeperClientBuilder.java | 209 --
.../DistributedLogConfiguration.java | 3528 ------------------
.../distributedlog/DistributedLogConstants.java | 69 -
.../distributedlog/DistributedLogManager.java | 308 --
.../java/com/twitter/distributedlog/Entry.java | 403 --
.../com/twitter/distributedlog/EntryBuffer.java | 70 -
.../twitter/distributedlog/EntryPosition.java | 63 -
.../twitter/distributedlog/EnvelopedEntry.java | 296 --
.../distributedlog/EnvelopedEntryReader.java | 102 -
.../distributedlog/EnvelopedEntryWriter.java | 192 -
.../distributedlog/LedgerReadPosition.java | 171 -
.../distributedlog/LocalDLMEmulator.java | 364 --
.../com/twitter/distributedlog/LogReader.java | 195 -
.../distributedlog/LogSegmentMetadata.java | 1125 ------
.../com/twitter/distributedlog/LogWriter.java | 78 -
.../distributedlog/MaxLogSegmentSequenceNo.java | 72 -
.../com/twitter/distributedlog/MaxTxId.java | 72 -
.../distributedlog/MetadataAccessor.java | 43 -
.../distributedlog/ReadAheadEntryReader.java | 992 -----
.../com/twitter/distributedlog/ReadUtils.java | 782 ----
.../twitter/distributedlog/WriteLimiter.java | 62 -
.../twitter/distributedlog/ZooKeeperClient.java | 402 --
.../distributedlog/ZooKeeperClientBuilder.java | 233 --
.../acl/AccessControlManager.java | 74 -
.../acl/DefaultAccessControlManager.java | 55 -
.../distributedlog/acl/package-info.java | 21 -
.../admin/DistributedLogAdmin.java | 921 -----
.../distributedlog/admin/package-info.java | 21 -
.../distributedlog/auditor/DLAuditor.java | 630 ----
.../bk/DynamicQuorumConfigProvider.java | 37 -
.../bk/ImmutableQuorumConfigProvider.java | 35 -
.../distributedlog/bk/LedgerAllocator.java | 32 -
.../bk/LedgerAllocatorDelegator.java | 83 -
.../distributedlog/bk/LedgerAllocatorPool.java | 458 ---
.../distributedlog/bk/LedgerAllocatorUtils.java | 54 -
.../twitter/distributedlog/bk/QuorumConfig.java | 92 -
.../distributedlog/bk/QuorumConfigProvider.java | 32 -
.../bk/SimpleLedgerAllocator.java | 536 ---
.../twitter/distributedlog/bk/package-info.java | 24 -
.../callback/LogSegmentListener.java | 42 -
.../callback/LogSegmentNamesListener.java | 42 -
.../callback/NamespaceListener.java | 34 -
.../callback/ReadAheadCallback.java | 25 -
.../distributedlog/callback/package-info.java | 21 -
.../config/ConcurrentBaseConfiguration.java | 76 -
.../config/ConcurrentConstConfiguration.java | 31 -
.../config/ConfigurationListener.java | 32 -
.../config/ConfigurationSubscription.java | 186 -
.../config/DynamicConfigurationFactory.java | 91 -
.../DynamicDistributedLogConfiguration.java | 356 --
.../config/FileConfigurationBuilder.java | 28 -
.../config/PropertiesConfigurationBuilder.java | 40 -
.../distributedlog/config/package-info.java | 21 -
.../distributedlog/exceptions/ZKException.java | 54 -
.../feature/AbstractFeatureProvider.java | 103 -
.../feature/ConfigurationFeatureProvider.java | 76 -
.../distributedlog/feature/CoreFeatureKeys.java | 29 -
.../feature/DefaultFeatureProvider.java | 47 -
.../DynamicConfigurationFeatureProvider.java | 132 -
.../distributedlog/feature/package-info.java | 21 -
.../function/CloseAsyncCloseableFunction.java | 51 -
.../function/DefaultValueMapFunction.java | 41 -
.../function/GetLastTxIdFunction.java | 43 -
.../function/GetVersionedValueFunction.java | 39 -
.../distributedlog/function/VoidFunctions.java | 34 -
.../distributedlog/function/package-info.java | 21 -
.../distributedlog/impl/BKNamespaceDriver.java | 631 ----
.../distributedlog/impl/ZKLogMetadataStore.java | 124 -
.../impl/ZKLogSegmentFilters.java | 89 -
.../impl/ZKLogSegmentMetadataStore.java | 503 ---
.../distributedlog/impl/ZKMetadataAccessor.java | 264 --
.../distributedlog/impl/ZKNamespaceWatcher.java | 133 -
.../impl/acl/ZKAccessControl.java | 232 --
.../impl/acl/ZKAccessControlManager.java | 374 --
.../federated/FederatedZKLogMetadataStore.java | 760 ----
.../impl/logsegment/BKLogSegmentAllocator.java | 85 -
.../logsegment/BKLogSegmentEntryReader.java | 837 -----
.../impl/logsegment/BKLogSegmentEntryStore.java | 286 --
.../logsegment/BKLogSegmentEntryWriter.java | 61 -
.../BKLogSegmentRandomAccessEntryReader.java | 119 -
.../distributedlog/impl/logsegment/BKUtils.java | 72 -
.../impl/metadata/BKDLConfig.java | 400 --
.../impl/metadata/ZKLogStreamMetadataStore.java | 633 ----
.../impl/metadata/ZkMetadataResolver.java | 72 -
.../distributedlog/impl/package-info.java | 21 -
.../subscription/ZKSubscriptionStateStore.java | 121 -
.../impl/subscription/ZKSubscriptionsStore.java | 164 -
.../injector/AsyncFailureInjector.java | 131 -
.../injector/AsyncRandomFailureInjector.java | 186 -
.../injector/FailureInjector.java | 37 -
.../injector/RandomDelayFailureInjector.java | 60 -
.../distributedlog/injector/package-info.java | 21 -
.../twitter/distributedlog/io/Abortable.java | 41 -
.../twitter/distributedlog/io/Abortables.java | 183 -
.../distributedlog/io/AsyncAbortable.java | 57 -
.../distributedlog/io/AsyncCloseable.java | 60 -
.../distributedlog/io/AsyncDeleteable.java | 34 -
.../twitter/distributedlog/io/package-info.java | 21 -
.../limiter/ChainedRequestLimiter.java | 76 -
.../limiter/ComposableRequestLimiter.java | 73 -
.../limiter/GuavaRateLimiter.java | 58 -
.../distributedlog/limiter/RateLimiter.java | 49 -
.../distributedlog/limiter/RequestLimiter.java | 24 -
.../distributedlog/limiter/package-info.java | 21 -
.../distributedlog/lock/DistributedLock.java | 54 -
.../lock/DistributedLockContext.java | 43 -
.../lock/EpochChangedException.java | 33 -
.../twitter/distributedlog/lock/LockAction.java | 36 -
.../lock/LockClosedException.java | 38 -
.../distributedlog/lock/LockListener.java | 28 -
.../lock/LockSessionExpiredException.java | 34 -
.../lock/LockStateChangedException.java | 36 -
.../lock/LockTimeoutException.java | 34 -
.../twitter/distributedlog/lock/LockWaiter.java | 95 -
.../distributedlog/lock/NopDistributedLock.java | 51 -
.../distributedlog/lock/SessionLock.java | 126 -
.../distributedlog/lock/SessionLockFactory.java | 38 -
.../distributedlog/lock/ZKDistributedLock.java | 537 ---
.../distributedlog/lock/ZKSessionLock.java | 1363 -------
.../lock/ZKSessionLockFactory.java | 133 -
.../distributedlog/lock/package-info.java | 21 -
.../logsegment/LogSegmentEntryReader.java | 113 -
.../logsegment/LogSegmentEntryStore.java | 73 -
.../logsegment/LogSegmentEntryWriter.java | 72 -
.../logsegment/LogSegmentFilter.java | 42 -
.../logsegment/LogSegmentMetadataCache.java | 98 -
.../logsegment/LogSegmentMetadataStore.java | 163 -
.../LogSegmentRandomAccessEntryReader.java | 47 -
.../logsegment/LogSegmentWriter.java | 87 -
.../logsegment/PerStreamLogSegmentCache.java | 243 --
.../logsegment/RollingPolicy.java | 33 -
.../logsegment/SizeBasedRollingPolicy.java | 34 -
.../logsegment/TimeBasedRollingPolicy.java | 46 -
.../distributedlog/logsegment/package-info.java | 21 -
.../distributedlog/metadata/DLConfig.java | 39 -
.../distributedlog/metadata/DLMetadata.java | 227 --
.../DryrunLogSegmentMetadataStoreUpdater.java | 51 -
.../distributedlog/metadata/LogMetadata.java | 175 -
.../metadata/LogMetadataForReader.java | 103 -
.../metadata/LogMetadataForWriter.java | 64 -
.../metadata/LogMetadataStore.java | 66 -
.../LogSegmentMetadataStoreUpdater.java | 184 -
.../metadata/LogStreamMetadataStore.java | 114 -
.../metadata/MetadataResolver.java | 51 -
.../metadata/MetadataUpdater.java | 120 -
.../distributedlog/metadata/package-info.java | 21 -
.../namespace/DistributedLogNamespace.java | 190 -
.../DistributedLogNamespaceBuilder.java | 278 --
.../namespace/NamespaceDriver.java | 138 -
.../namespace/NamespaceDriverManager.java | 180 -
.../namespace/NamespaceWatcher.java | 61 -
.../distributedlog/namespace/package-info.java | 21 -
.../twitter/distributedlog/net/DNSResolver.java | 113 -
.../distributedlog/net/DNSResolverForRacks.java | 66 -
.../distributedlog/net/DNSResolverForRows.java | 72 -
.../twitter/distributedlog/net/NetUtils.java | 74 -
.../distributedlog/net/package-info.java | 33 -
.../twitter/distributedlog/package-info.java | 21 -
.../distributedlog/rate/MovingAverageRate.java | 24 -
.../rate/MovingAverageRateFactory.java | 65 -
.../rate/SampledMovingAverageRate.java | 58 -
.../distributedlog/readahead/package-info.java | 21 -
.../selector/FirstDLSNNotLessThanSelector.java | 46 -
.../selector/FirstRecordSelector.java | 46 -
.../selector/FirstTxIdNotLessThanSelector.java | 51 -
.../selector/LastRecordSelector.java | 38 -
.../selector/LogRecordSelector.java | 40 -
.../stats/BKExceptionStatsLogger.java | 109 -
.../stats/BroadCastStatsLogger.java | 194 -
.../distributedlog/stats/OpStatsListener.java | 51 -
.../subscription/SubscriptionStateStore.java | 42 -
.../subscription/SubscriptionsStore.java | 69 -
.../tools/DistributedLogTool.java | 2873 --------------
.../com/twitter/distributedlog/tools/Tool.java | 243 --
.../distributedlog/tools/package-info.java | 21 -
.../twitter/distributedlog/util/Allocator.java | 102 -
.../distributedlog/util/CommandLineUtils.java | 56 -
.../twitter/distributedlog/util/ConfUtils.java | 65 -
.../twitter/distributedlog/util/DLUtils.java | 321 --
.../distributedlog/util/FailpointUtils.java | 134 -
.../distributedlog/util/FutureUtils.java | 534 ---
.../util/MonitoredFuturePool.java | 131 -
.../MonitoredScheduledThreadPoolExecutor.java | 257 --
.../distributedlog/util/OrderedScheduler.java | 490 ---
.../distributedlog/util/PermitLimiter.java | 57 -
.../distributedlog/util/PermitManager.java | 93 -
.../distributedlog/util/RetryPolicyUtils.java | 45 -
.../util/SafeQueueingFuturePool.java | 115 -
.../distributedlog/util/SchedulerUtils.java | 56 -
.../twitter/distributedlog/util/Sequencer.java | 31 -
.../util/SimplePermitLimiter.java | 117 -
.../twitter/distributedlog/util/Sizable.java | 31 -
.../distributedlog/util/TimeSequencer.java | 39 -
.../distributedlog/util/Transaction.java | 97 -
.../com/twitter/distributedlog/util/Utils.java | 607 ---
.../distributedlog/util/package-info.java | 21 -
.../twitter/distributedlog/zk/DefaultZKOp.java | 55 -
.../distributedlog/zk/LimitedPermitManager.java | 195 -
.../com/twitter/distributedlog/zk/ZKOp.java | 63 -
.../distributedlog/zk/ZKTransaction.java | 103 -
.../distributedlog/zk/ZKVersionedSetOp.java | 71 -
.../distributedlog/zk/ZKWatcherManager.java | 239 --
.../distributedlog/AppendOnlyStreamReader.java | 198 +
.../distributedlog/AppendOnlyStreamWriter.java | 107 +
.../apache/distributedlog/AsyncLogReader.java | 69 +
.../apache/distributedlog/AsyncLogWriter.java | 70 +
.../distributedlog/AsyncNotification.java | 32 +
.../distributedlog/BKAbstractLogWriter.java | 555 +++
.../apache/distributedlog/BKAsyncLogReader.java | 751 ++++
.../apache/distributedlog/BKAsyncLogWriter.java | 559 +++
.../distributedlog/BKDistributedLogManager.java | 1106 ++++++
.../BKDistributedLogNamespace.java | 320 ++
.../org/apache/distributedlog/BKLogHandler.java | 715 ++++
.../apache/distributedlog/BKLogReadHandler.java | 431 +++
.../distributedlog/BKLogSegmentWriter.java | 1348 +++++++
.../distributedlog/BKLogWriteHandler.java | 1325 +++++++
.../apache/distributedlog/BKSyncLogReader.java | 276 ++
.../apache/distributedlog/BKSyncLogWriter.java | 113 +
.../apache/distributedlog/BKTransmitPacket.java | 90 +
.../apache/distributedlog/BookKeeperClient.java | 289 ++
.../distributedlog/BookKeeperClientBuilder.java | 209 ++
.../DistributedLogConfiguration.java | 3528 ++++++++++++++++++
.../distributedlog/DistributedLogConstants.java | 69 +
.../distributedlog/DistributedLogManager.java | 308 ++
.../java/org/apache/distributedlog/Entry.java | 403 ++
.../org/apache/distributedlog/EntryBuffer.java | 70 +
.../apache/distributedlog/EntryPosition.java | 63 +
.../apache/distributedlog/EnvelopedEntry.java | 296 ++
.../distributedlog/EnvelopedEntryReader.java | 102 +
.../distributedlog/EnvelopedEntryWriter.java | 192 +
.../distributedlog/LedgerReadPosition.java | 171 +
.../apache/distributedlog/LocalDLMEmulator.java | 364 ++
.../org/apache/distributedlog/LogReader.java | 195 +
.../distributedlog/LogSegmentMetadata.java | 1125 ++++++
.../org/apache/distributedlog/LogWriter.java | 78 +
.../distributedlog/MaxLogSegmentSequenceNo.java | 72 +
.../java/org/apache/distributedlog/MaxTxId.java | 72 +
.../apache/distributedlog/MetadataAccessor.java | 43 +
.../distributedlog/ReadAheadEntryReader.java | 992 +++++
.../org/apache/distributedlog/ReadUtils.java | 782 ++++
.../org/apache/distributedlog/WriteLimiter.java | 62 +
.../apache/distributedlog/ZooKeeperClient.java | 402 ++
.../distributedlog/ZooKeeperClientBuilder.java | 233 ++
.../acl/AccessControlManager.java | 74 +
.../acl/DefaultAccessControlManager.java | 55 +
.../apache/distributedlog/acl/package-info.java | 21 +
.../admin/DistributedLogAdmin.java | 921 +++++
.../distributedlog/admin/package-info.java | 21 +
.../distributedlog/auditor/DLAuditor.java | 630 ++++
.../bk/DynamicQuorumConfigProvider.java | 37 +
.../bk/ImmutableQuorumConfigProvider.java | 35 +
.../distributedlog/bk/LedgerAllocator.java | 32 +
.../bk/LedgerAllocatorDelegator.java | 83 +
.../distributedlog/bk/LedgerAllocatorPool.java | 458 +++
.../distributedlog/bk/LedgerAllocatorUtils.java | 54 +
.../apache/distributedlog/bk/QuorumConfig.java | 92 +
.../distributedlog/bk/QuorumConfigProvider.java | 32 +
.../bk/SimpleLedgerAllocator.java | 536 +++
.../apache/distributedlog/bk/package-info.java | 24 +
.../callback/LogSegmentListener.java | 42 +
.../callback/LogSegmentNamesListener.java | 42 +
.../callback/NamespaceListener.java | 34 +
.../callback/ReadAheadCallback.java | 25 +
.../distributedlog/callback/package-info.java | 21 +
.../config/ConcurrentBaseConfiguration.java | 76 +
.../config/ConcurrentConstConfiguration.java | 31 +
.../config/ConfigurationListener.java | 32 +
.../config/ConfigurationSubscription.java | 186 +
.../config/DynamicConfigurationFactory.java | 91 +
.../DynamicDistributedLogConfiguration.java | 356 ++
.../config/FileConfigurationBuilder.java | 28 +
.../config/PropertiesConfigurationBuilder.java | 40 +
.../distributedlog/config/package-info.java | 21 +
.../distributedlog/exceptions/ZKException.java | 54 +
.../feature/AbstractFeatureProvider.java | 103 +
.../feature/ConfigurationFeatureProvider.java | 76 +
.../distributedlog/feature/CoreFeatureKeys.java | 29 +
.../feature/DefaultFeatureProvider.java | 47 +
.../DynamicConfigurationFeatureProvider.java | 132 +
.../distributedlog/feature/package-info.java | 21 +
.../function/CloseAsyncCloseableFunction.java | 51 +
.../function/DefaultValueMapFunction.java | 41 +
.../function/GetLastTxIdFunction.java | 43 +
.../function/GetVersionedValueFunction.java | 39 +
.../distributedlog/function/VoidFunctions.java | 34 +
.../distributedlog/function/package-info.java | 21 +
.../distributedlog/impl/BKNamespaceDriver.java | 631 ++++
.../distributedlog/impl/ZKLogMetadataStore.java | 124 +
.../impl/ZKLogSegmentFilters.java | 89 +
.../impl/ZKLogSegmentMetadataStore.java | 503 +++
.../distributedlog/impl/ZKMetadataAccessor.java | 264 ++
.../distributedlog/impl/ZKNamespaceWatcher.java | 133 +
.../impl/acl/ZKAccessControl.java | 232 ++
.../impl/acl/ZKAccessControlManager.java | 374 ++
.../federated/FederatedZKLogMetadataStore.java | 760 ++++
.../impl/logsegment/BKLogSegmentAllocator.java | 85 +
.../logsegment/BKLogSegmentEntryReader.java | 837 +++++
.../impl/logsegment/BKLogSegmentEntryStore.java | 286 ++
.../logsegment/BKLogSegmentEntryWriter.java | 61 +
.../BKLogSegmentRandomAccessEntryReader.java | 119 +
.../distributedlog/impl/logsegment/BKUtils.java | 72 +
.../impl/metadata/BKDLConfig.java | 400 ++
.../impl/metadata/ZKLogStreamMetadataStore.java | 633 ++++
.../impl/metadata/ZkMetadataResolver.java | 72 +
.../distributedlog/impl/package-info.java | 21 +
.../subscription/ZKSubscriptionStateStore.java | 121 +
.../impl/subscription/ZKSubscriptionsStore.java | 164 +
.../injector/AsyncFailureInjector.java | 131 +
.../injector/AsyncRandomFailureInjector.java | 186 +
.../injector/FailureInjector.java | 37 +
.../injector/RandomDelayFailureInjector.java | 60 +
.../distributedlog/injector/package-info.java | 21 +
.../org/apache/distributedlog/io/Abortable.java | 41 +
.../apache/distributedlog/io/Abortables.java | 183 +
.../distributedlog/io/AsyncAbortable.java | 57 +
.../distributedlog/io/AsyncCloseable.java | 60 +
.../distributedlog/io/AsyncDeleteable.java | 34 +
.../apache/distributedlog/io/package-info.java | 21 +
.../limiter/ChainedRequestLimiter.java | 76 +
.../limiter/ComposableRequestLimiter.java | 73 +
.../limiter/GuavaRateLimiter.java | 58 +
.../distributedlog/limiter/RateLimiter.java | 49 +
.../distributedlog/limiter/RequestLimiter.java | 24 +
.../distributedlog/limiter/package-info.java | 21 +
.../distributedlog/lock/DistributedLock.java | 54 +
.../lock/DistributedLockContext.java | 43 +
.../lock/EpochChangedException.java | 33 +
.../apache/distributedlog/lock/LockAction.java | 36 +
.../lock/LockClosedException.java | 38 +
.../distributedlog/lock/LockListener.java | 28 +
.../lock/LockSessionExpiredException.java | 34 +
.../lock/LockStateChangedException.java | 36 +
.../lock/LockTimeoutException.java | 34 +
.../apache/distributedlog/lock/LockWaiter.java | 95 +
.../distributedlog/lock/NopDistributedLock.java | 51 +
.../apache/distributedlog/lock/SessionLock.java | 126 +
.../distributedlog/lock/SessionLockFactory.java | 38 +
.../distributedlog/lock/ZKDistributedLock.java | 537 +++
.../distributedlog/lock/ZKSessionLock.java | 1363 +++++++
.../lock/ZKSessionLockFactory.java | 133 +
.../distributedlog/lock/package-info.java | 21 +
.../logsegment/LogSegmentEntryReader.java | 113 +
.../logsegment/LogSegmentEntryStore.java | 73 +
.../logsegment/LogSegmentEntryWriter.java | 72 +
.../logsegment/LogSegmentFilter.java | 42 +
.../logsegment/LogSegmentMetadataCache.java | 98 +
.../logsegment/LogSegmentMetadataStore.java | 163 +
.../LogSegmentRandomAccessEntryReader.java | 47 +
.../logsegment/LogSegmentWriter.java | 87 +
.../logsegment/PerStreamLogSegmentCache.java | 243 ++
.../logsegment/RollingPolicy.java | 33 +
.../logsegment/SizeBasedRollingPolicy.java | 34 +
.../logsegment/TimeBasedRollingPolicy.java | 46 +
.../distributedlog/logsegment/package-info.java | 21 +
.../distributedlog/metadata/DLConfig.java | 39 +
.../distributedlog/metadata/DLMetadata.java | 227 ++
.../DryrunLogSegmentMetadataStoreUpdater.java | 51 +
.../distributedlog/metadata/LogMetadata.java | 175 +
.../metadata/LogMetadataForReader.java | 103 +
.../metadata/LogMetadataForWriter.java | 64 +
.../metadata/LogMetadataStore.java | 66 +
.../LogSegmentMetadataStoreUpdater.java | 184 +
.../metadata/LogStreamMetadataStore.java | 114 +
.../metadata/MetadataResolver.java | 51 +
.../metadata/MetadataUpdater.java | 120 +
.../distributedlog/metadata/package-info.java | 21 +
.../namespace/DistributedLogNamespace.java | 190 +
.../DistributedLogNamespaceBuilder.java | 278 ++
.../namespace/NamespaceDriver.java | 138 +
.../namespace/NamespaceDriverManager.java | 180 +
.../namespace/NamespaceWatcher.java | 61 +
.../distributedlog/namespace/package-info.java | 21 +
.../apache/distributedlog/net/DNSResolver.java | 113 +
.../distributedlog/net/DNSResolverForRacks.java | 66 +
.../distributedlog/net/DNSResolverForRows.java | 72 +
.../org/apache/distributedlog/net/NetUtils.java | 74 +
.../apache/distributedlog/net/package-info.java | 33 +
.../org/apache/distributedlog/package-info.java | 21 +
.../distributedlog/rate/MovingAverageRate.java | 24 +
.../rate/MovingAverageRateFactory.java | 65 +
.../rate/SampledMovingAverageRate.java | 58 +
.../distributedlog/readahead/package-info.java | 21 +
.../selector/FirstDLSNNotLessThanSelector.java | 46 +
.../selector/FirstRecordSelector.java | 46 +
.../selector/FirstTxIdNotLessThanSelector.java | 51 +
.../selector/LastRecordSelector.java | 38 +
.../selector/LogRecordSelector.java | 40 +
.../stats/BKExceptionStatsLogger.java | 109 +
.../stats/BroadCastStatsLogger.java | 194 +
.../distributedlog/stats/OpStatsListener.java | 51 +
.../subscription/SubscriptionStateStore.java | 42 +
.../subscription/SubscriptionsStore.java | 69 +
.../tools/DistributedLogTool.java | 2873 ++++++++++++++
.../org/apache/distributedlog/tools/Tool.java | 243 ++
.../distributedlog/tools/package-info.java | 21 +
.../apache/distributedlog/util/Allocator.java | 102 +
.../distributedlog/util/CommandLineUtils.java | 56 +
.../apache/distributedlog/util/ConfUtils.java | 65 +
.../org/apache/distributedlog/util/DLUtils.java | 321 ++
.../distributedlog/util/FailpointUtils.java | 134 +
.../apache/distributedlog/util/FutureUtils.java | 534 +++
.../util/MonitoredFuturePool.java | 131 +
.../MonitoredScheduledThreadPoolExecutor.java | 257 ++
.../distributedlog/util/OrderedScheduler.java | 490 +++
.../distributedlog/util/PermitLimiter.java | 57 +
.../distributedlog/util/PermitManager.java | 93 +
.../distributedlog/util/RetryPolicyUtils.java | 45 +
.../util/SafeQueueingFuturePool.java | 115 +
.../distributedlog/util/SchedulerUtils.java | 56 +
.../apache/distributedlog/util/Sequencer.java | 31 +
.../util/SimplePermitLimiter.java | 117 +
.../org/apache/distributedlog/util/Sizable.java | 31 +
.../distributedlog/util/TimeSequencer.java | 39 +
.../apache/distributedlog/util/Transaction.java | 97 +
.../org/apache/distributedlog/util/Utils.java | 607 +++
.../distributedlog/util/package-info.java | 21 +
.../apache/distributedlog/zk/DefaultZKOp.java | 55 +
.../distributedlog/zk/LimitedPermitManager.java | 195 +
.../java/org/apache/distributedlog/zk/ZKOp.java | 63 +
.../apache/distributedlog/zk/ZKTransaction.java | 103 +
.../distributedlog/zk/ZKVersionedSetOp.java | 71 +
.../distributedlog/zk/ZKWatcherManager.java | 239 ++
.../src/main/resources/findbugsExclude.xml | 8 +-
.../src/main/thrift/metadata.thrift | 2 +-
.../com/twitter/distributedlog/DLMTestUtil.java | 489 ---
.../NonBlockingReadsTestUtil.java | 145 -
.../TestAppendOnlyStreamReader.java | 207 -
.../TestAppendOnlyStreamWriter.java | 337 --
.../distributedlog/TestAsyncBulkWrite.java | 351 --
.../distributedlog/TestAsyncReaderLock.java | 607 ---
.../distributedlog/TestAsyncReaderWriter.java | 2206 -----------
.../TestBKDistributedLogManager.java | 1201 ------
.../TestBKDistributedLogNamespace.java | 440 ---
.../distributedlog/TestBKLogReadHandler.java | 398 --
.../distributedlog/TestBKLogSegmentWriter.java | 790 ----
.../distributedlog/TestBKLogWriteHandler.java | 90 -
.../distributedlog/TestBKSyncLogReader.java | 306 --
.../twitter/distributedlog/TestDLMTestUtil.java | 62 -
.../distributedlog/TestDistributedLogBase.java | 244 --
.../TestDistributedLogConfiguration.java | 132 -
.../com/twitter/distributedlog/TestEntry.java | 345 --
.../distributedlog/TestEntryPosition.java | 59 -
.../distributedlog/TestEnvelopedEntry.java | 81 -
.../distributedlog/TestInterleavedReaders.java | 338 --
.../distributedlog/TestLogSegmentCreation.java | 106 -
.../distributedlog/TestLogSegmentMetadata.java | 159 -
.../distributedlog/TestLogSegmentsZK.java | 245 --
.../distributedlog/TestNonBlockingReads.java | 348 --
.../TestNonBlockingReadsMultiReader.java | 168 -
.../TestReadAheadEntryReader.java | 463 ---
.../twitter/distributedlog/TestReadUtils.java | 344 --
.../com/twitter/distributedlog/TestReader.java | 206 -
.../distributedlog/TestRollLogSegments.java | 429 ---
.../twitter/distributedlog/TestSequenceID.java | 254 --
.../twitter/distributedlog/TestTruncate.java | 341 --
.../distributedlog/TestWriteLimiter.java | 205 -
.../distributedlog/TestZooKeeperClient.java | 449 ---
.../TestZooKeeperClientBuilder.java | 57 -
.../distributedlog/ZooKeeperClientUtils.java | 95 -
.../ZooKeeperClusterTestCase.java | 53 -
.../distributedlog/acl/TestZKAccessControl.java | 154 -
.../acl/TestZKAccessControlManager.java | 178 -
.../twitter/distributedlog/admin/TestDLCK.java | 164 -
.../admin/TestDistributedLogAdmin.java | 194 -
.../distributedlog/bk/TestLedgerAllocator.java | 389 --
.../bk/TestLedgerAllocatorPool.java | 311 --
.../distributedlog/config/PropertiesWriter.java | 67 -
.../config/TestConcurrentBaseConfiguration.java | 46 -
.../config/TestConfigurationSubscription.java | 171 -
.../config/TestDynamicConfigurationFactory.java | 101 -
.../TestDynamicDistributedLogConfiguration.java | 261 --
.../TestConfigurationFeatureProvider.java | 65 -
...TestDynamicConfigurationFeatureProvider.java | 183 -
.../impl/TestZKLogMetadataStore.java | 114 -
.../impl/TestZKLogSegmentFilters.java | 74 -
.../impl/TestZKLogSegmentMetadataStore.java | 831 -----
.../impl/TestZKNamespaceWatcher.java | 185 -
.../TestFederatedZKLogMetadataStore.java | 446 ---
.../logsegment/TestBKLogSegmentEntryReader.java | 561 ---
.../metadata/TestZKLogStreamMetadataStore.java | 327 --
.../TestZKLogStreamMetadataStoreUtils.java | 208 --
.../impl/metadata/TestZkMetadataResolver.java | 203 -
.../limiter/TestRequestLimiter.java | 60 -
.../lock/TestDistributedLock.java | 835 -----
.../distributedlog/lock/TestZKSessionLock.java | 1224 ------
.../TestPerStreamLogSegmentCache.java | 186 -
.../logsegment/TestRollingPolicy.java | 63 -
.../distributedlog/metadata/TestDLMetadata.java | 195 -
.../metadata/TestLogMetadata.java | 59 -
.../TestLogSegmentMetadataStoreUpdater.java | 284 --
.../TestDistributedLogNamespaceBuilder.java | 124 -
.../distributedlog/net/TestDNSResolver.java | 96 -
.../distributedlog/net/TestNetUtils.java | 82 -
.../rate/TestMovingAverageRate.java | 99 -
.../selector/TestLogRecordSelectors.java | 128 -
.../tools/TestDistributedLogTool.java | 231 --
.../distributedlog/util/TestConfUtils.java | 53 -
.../distributedlog/util/TestDLUtils.java | 273 --
.../distributedlog/util/TestFutureUtils.java | 71 -
.../distributedlog/util/TestPermitManager.java | 92 -
.../util/TestSafeQueueingFuturePool.java | 205 -
.../distributedlog/util/TestTimeSequencer.java | 42 -
.../twitter/distributedlog/util/TestUtils.java | 125 -
.../distributedlog/zk/TestZKTransaction.java | 97 -
.../distributedlog/zk/TestZKVersionedSetOp.java | 86 -
.../distributedlog/zk/TestZKWatcherManager.java | 80 -
.../org/apache/distributedlog/DLMTestUtil.java | 489 +++
.../NonBlockingReadsTestUtil.java | 145 +
.../TestAppendOnlyStreamReader.java | 207 +
.../TestAppendOnlyStreamWriter.java | 337 ++
.../distributedlog/TestAsyncBulkWrite.java | 351 ++
.../distributedlog/TestAsyncReaderLock.java | 607 +++
.../distributedlog/TestAsyncReaderWriter.java | 2206 +++++++++++
.../TestBKDistributedLogManager.java | 1201 ++++++
.../TestBKDistributedLogNamespace.java | 440 +++
.../distributedlog/TestBKLogReadHandler.java | 398 ++
.../distributedlog/TestBKLogSegmentWriter.java | 790 ++++
.../distributedlog/TestBKLogWriteHandler.java | 90 +
.../distributedlog/TestBKSyncLogReader.java | 306 ++
.../apache/distributedlog/TestDLMTestUtil.java | 62 +
.../distributedlog/TestDistributedLogBase.java | 244 ++
.../TestDistributedLogConfiguration.java | 132 +
.../org/apache/distributedlog/TestEntry.java | 345 ++
.../distributedlog/TestEntryPosition.java | 59 +
.../distributedlog/TestEnvelopedEntry.java | 81 +
.../distributedlog/TestInterleavedReaders.java | 338 ++
.../distributedlog/TestLogSegmentCreation.java | 106 +
.../distributedlog/TestLogSegmentMetadata.java | 159 +
.../distributedlog/TestLogSegmentsZK.java | 245 ++
.../distributedlog/TestNonBlockingReads.java | 348 ++
.../TestNonBlockingReadsMultiReader.java | 168 +
.../TestReadAheadEntryReader.java | 463 +++
.../apache/distributedlog/TestReadUtils.java | 344 ++
.../org/apache/distributedlog/TestReader.java | 206 +
.../distributedlog/TestRollLogSegments.java | 429 +++
.../apache/distributedlog/TestSequenceID.java | 254 ++
.../org/apache/distributedlog/TestTruncate.java | 341 ++
.../apache/distributedlog/TestWriteLimiter.java | 205 +
.../distributedlog/TestZooKeeperClient.java | 449 +++
.../TestZooKeeperClientBuilder.java | 57 +
.../distributedlog/ZooKeeperClientUtils.java | 95 +
.../ZooKeeperClusterTestCase.java | 53 +
.../distributedlog/acl/TestZKAccessControl.java | 154 +
.../acl/TestZKAccessControlManager.java | 178 +
.../apache/distributedlog/admin/TestDLCK.java | 164 +
.../admin/TestDistributedLogAdmin.java | 194 +
.../distributedlog/bk/TestLedgerAllocator.java | 389 ++
.../bk/TestLedgerAllocatorPool.java | 311 ++
.../distributedlog/config/PropertiesWriter.java | 67 +
.../config/TestConcurrentBaseConfiguration.java | 46 +
.../config/TestConfigurationSubscription.java | 171 +
.../config/TestDynamicConfigurationFactory.java | 101 +
.../TestDynamicDistributedLogConfiguration.java | 261 ++
.../TestConfigurationFeatureProvider.java | 65 +
...TestDynamicConfigurationFeatureProvider.java | 183 +
.../impl/TestZKLogMetadataStore.java | 114 +
.../impl/TestZKLogSegmentFilters.java | 74 +
.../impl/TestZKLogSegmentMetadataStore.java | 831 +++++
.../impl/TestZKNamespaceWatcher.java | 185 +
.../TestFederatedZKLogMetadataStore.java | 446 +++
.../logsegment/TestBKLogSegmentEntryReader.java | 561 +++
.../metadata/TestZKLogStreamMetadataStore.java | 327 ++
.../TestZKLogStreamMetadataStoreUtils.java | 208 ++
.../impl/metadata/TestZkMetadataResolver.java | 203 +
.../limiter/TestRequestLimiter.java | 60 +
.../lock/TestDistributedLock.java | 835 +++++
.../distributedlog/lock/TestZKSessionLock.java | 1224 ++++++
.../TestPerStreamLogSegmentCache.java | 186 +
.../logsegment/TestRollingPolicy.java | 63 +
.../distributedlog/metadata/TestDLMetadata.java | 195 +
.../metadata/TestLogMetadata.java | 59 +
.../TestLogSegmentMetadataStoreUpdater.java | 284 ++
.../TestDistributedLogNamespaceBuilder.java | 124 +
.../distributedlog/net/TestDNSResolver.java | 96 +
.../apache/distributedlog/net/TestNetUtils.java | 82 +
.../rate/TestMovingAverageRate.java | 99 +
.../selector/TestLogRecordSelectors.java | 128 +
.../tools/TestDistributedLogTool.java | 231 ++
.../distributedlog/util/TestConfUtils.java | 53 +
.../apache/distributedlog/util/TestDLUtils.java | 273 ++
.../distributedlog/util/TestFutureUtils.java | 71 +
.../distributedlog/util/TestPermitManager.java | 92 +
.../util/TestSafeQueueingFuturePool.java | 205 +
.../distributedlog/util/TestTimeSequencer.java | 42 +
.../apache/distributedlog/util/TestUtils.java | 125 +
.../distributedlog/zk/TestZKTransaction.java | 97 +
.../distributedlog/zk/TestZKVersionedSetOp.java | 86 +
.../distributedlog/zk/TestZKWatcherManager.java | 80 +
.../java/com/twitter/distributedlog/DLSN.java | 266 --
.../EnvelopedRecordSetReader.java | 118 -
.../EnvelopedRecordSetWriter.java | 197 -
.../com/twitter/distributedlog/LogRecord.java | 606 ---
.../twitter/distributedlog/LogRecordSet.java | 155 -
.../distributedlog/LogRecordSetBuffer.java | 65 -
.../distributedlog/LogRecordWithDLSN.java | 107 -
.../twitter/distributedlog/RecordStream.java | 42 -
.../annotations/DistributedLogAnnotations.java | 35 -
.../annotations/package-info.java | 21 -
.../exceptions/AlreadyClosedException.java | 34 -
.../AlreadyTruncatedTransactionException.java | 32 -
.../exceptions/BKTransmitException.java | 42 -
.../exceptions/ChecksumFailedException.java | 32 -
.../exceptions/DLClientClosedException.java | 36 -
.../distributedlog/exceptions/DLException.java | 81 -
.../exceptions/DLIllegalStateException.java | 36 -
.../exceptions/DLInterruptedException.java | 36 -
.../exceptions/EndOfLogSegmentException.java | 32 -
.../exceptions/EndOfStreamException.java | 32 -
.../exceptions/FlushException.java | 51 -
.../exceptions/IdleReaderException.java | 34 -
.../exceptions/InternalServerException.java | 40 -
.../InvalidEnvelopedEntryException.java | 32 -
.../exceptions/InvalidStreamNameException.java | 36 -
.../exceptions/LockCancelledException.java | 31 -
.../exceptions/LockingException.java | 44 -
.../exceptions/LogEmptyException.java | 32 -
.../exceptions/LogExistsException.java | 31 -
.../exceptions/LogNotFoundException.java | 32 -
.../exceptions/LogReadException.java | 33 -
.../exceptions/LogRecordTooLongException.java | 34 -
.../LogSegmentIsTruncatedException.java | 33 -
.../exceptions/LogSegmentNotFoundException.java | 32 -
.../exceptions/MetadataException.java | 36 -
.../exceptions/NotYetImplementedException.java | 32 -
.../exceptions/OverCapacityException.java | 39 -
.../OwnershipAcquireFailedException.java | 40 -
.../exceptions/ReadCancelledException.java | 33 -
.../exceptions/RegionUnavailableException.java | 34 -
.../exceptions/RequestDeniedException.java | 33 -
.../exceptions/RetryableReadException.java | 36 -
.../exceptions/ServiceUnavailableException.java | 32 -
.../exceptions/StreamNotReadyException.java | 32 -
.../exceptions/StreamUnavailableException.java | 32 -
.../exceptions/TooManyStreamsException.java | 32 -
.../TransactionIdOutOfOrderException.java | 47 -
.../exceptions/UnexpectedException.java | 40 -
.../UnsupportedMetadataVersionException.java | 32 -
.../exceptions/WriteCancelledException.java | 45 -
.../exceptions/WriteException.java | 33 -
.../distributedlog/exceptions/package-info.java | 21 -
.../com/twitter/distributedlog/io/Buffer.java | 33 -
.../distributedlog/io/CompressionCodec.java | 80 -
.../distributedlog/io/CompressionUtils.java | 58 -
.../io/IdentityCompressionCodec.java | 50 -
.../distributedlog/io/LZ4CompressionCodec.java | 101 -
.../distributedlog/io/TransmitListener.java | 56 -
.../twitter/distributedlog/io/package-info.java | 21 -
.../twitter/distributedlog/package-info.java | 21 -
.../distributedlog/util/BitMaskUtils.java | 57 -
.../distributedlog/util/ProtocolUtils.java | 79 -
.../distributedlog/util/package-info.java | 21 -
.../java/org/apache/distributedlog/DLSN.java | 266 ++
.../EnvelopedRecordSetReader.java | 118 +
.../EnvelopedRecordSetWriter.java | 197 +
.../org/apache/distributedlog/LogRecord.java | 606 +++
.../org/apache/distributedlog/LogRecordSet.java | 155 +
.../distributedlog/LogRecordSetBuffer.java | 65 +
.../distributedlog/LogRecordWithDLSN.java | 107 +
.../org/apache/distributedlog/RecordStream.java | 42 +
.../annotations/DistributedLogAnnotations.java | 35 +
.../annotations/package-info.java | 21 +
.../exceptions/AlreadyClosedException.java | 34 +
.../AlreadyTruncatedTransactionException.java | 32 +
.../exceptions/BKTransmitException.java | 42 +
.../exceptions/ChecksumFailedException.java | 32 +
.../exceptions/DLClientClosedException.java | 36 +
.../distributedlog/exceptions/DLException.java | 81 +
.../exceptions/DLIllegalStateException.java | 36 +
.../exceptions/DLInterruptedException.java | 36 +
.../exceptions/EndOfLogSegmentException.java | 32 +
.../exceptions/EndOfStreamException.java | 32 +
.../exceptions/FlushException.java | 51 +
.../exceptions/IdleReaderException.java | 34 +
.../exceptions/InternalServerException.java | 40 +
.../InvalidEnvelopedEntryException.java | 32 +
.../exceptions/InvalidStreamNameException.java | 36 +
.../exceptions/LockCancelledException.java | 31 +
.../exceptions/LockingException.java | 44 +
.../exceptions/LogEmptyException.java | 32 +
.../exceptions/LogExistsException.java | 31 +
.../exceptions/LogNotFoundException.java | 32 +
.../exceptions/LogReadException.java | 33 +
.../exceptions/LogRecordTooLongException.java | 34 +
.../LogSegmentIsTruncatedException.java | 33 +
.../exceptions/LogSegmentNotFoundException.java | 32 +
.../exceptions/MetadataException.java | 36 +
.../exceptions/NotYetImplementedException.java | 32 +
.../exceptions/OverCapacityException.java | 39 +
.../OwnershipAcquireFailedException.java | 40 +
.../exceptions/ReadCancelledException.java | 33 +
.../exceptions/RegionUnavailableException.java | 34 +
.../exceptions/RequestDeniedException.java | 33 +
.../exceptions/RetryableReadException.java | 36 +
.../exceptions/ServiceUnavailableException.java | 32 +
.../exceptions/StreamNotReadyException.java | 32 +
.../exceptions/StreamUnavailableException.java | 32 +
.../exceptions/TooManyStreamsException.java | 32 +
.../TransactionIdOutOfOrderException.java | 47 +
.../exceptions/UnexpectedException.java | 40 +
.../UnsupportedMetadataVersionException.java | 32 +
.../exceptions/WriteCancelledException.java | 45 +
.../exceptions/WriteException.java | 33 +
.../distributedlog/exceptions/package-info.java | 21 +
.../org/apache/distributedlog/io/Buffer.java | 33 +
.../distributedlog/io/CompressionCodec.java | 80 +
.../distributedlog/io/CompressionUtils.java | 58 +
.../io/IdentityCompressionCodec.java | 50 +
.../distributedlog/io/LZ4CompressionCodec.java | 101 +
.../distributedlog/io/TransmitListener.java | 56 +
.../apache/distributedlog/io/package-info.java | 21 +
.../org/apache/distributedlog/package-info.java | 21 +
.../distributedlog/util/BitMaskUtils.java | 57 +
.../distributedlog/util/ProtocolUtils.java | 79 +
.../distributedlog/util/package-info.java | 21 +
.../src/main/resources/findbugsExclude.xml | 8 +-
.../src/main/thrift/service.thrift | 2 +-
.../com/twitter/distributedlog/TestDLSN.java | 64 -
.../distributedlog/TestLogRecordSet.java | 178 -
.../TestTimedOutTestsListener.java | 183 -
.../distributedlog/TimedOutTestsListener.java | 168 -
.../org/apache/distributedlog/TestDLSN.java | 64 +
.../apache/distributedlog/TestLogRecordSet.java | 178 +
.../TestTimedOutTestsListener.java | 183 +
.../distributedlog/TimedOutTestsListener.java | 168 +
distributedlog-service/bin/dlog-daemon.sh | 2 +-
distributedlog-service/bin/dlog-start.sh | 2 +-
distributedlog-service/pom.xml | 2 +-
.../distributedlog/service/ClientUtils.java | 33 -
.../service/DistributedLogCluster.java | 352 --
.../service/DistributedLogServer.java | 460 ---
.../service/DistributedLogServerApp.java | 187 -
.../service/DistributedLogServiceImpl.java | 794 ----
.../service/FatalErrorHandler.java | 30 -
.../distributedlog/service/MonitorService.java | 469 ---
.../service/MonitorServiceApp.java | 133 -
.../distributedlog/service/ResponseUtils.java | 86 -
.../service/ServerFeatureKeys.java | 29 -
.../distributedlog/service/StatsFilter.java | 60 -
.../service/announcer/Announcer.java | 41 -
.../service/announcer/NOPAnnouncer.java | 40 -
.../service/announcer/ServerSetAnnouncer.java | 111 -
.../service/announcer/package-info.java | 21 -
.../service/balancer/Balancer.java | 68 -
.../service/balancer/BalancerTool.java | 327 --
.../service/balancer/BalancerUtils.java | 74 -
.../service/balancer/ClusterBalancer.java | 378 --
.../balancer/CountBasedStreamChooser.java | 109 -
.../service/balancer/LimitedStreamChooser.java | 57 -
.../service/balancer/SimpleBalancer.java | 246 --
.../service/balancer/StreamChooser.java | 30 -
.../service/balancer/StreamMover.java | 34 -
.../service/balancer/StreamMoverImpl.java | 94 -
.../service/balancer/package-info.java | 21 -
.../config/DefaultStreamConfigProvider.java | 73 -
.../config/NullStreamConfigProvider.java | 40 -
.../service/config/ServerConfiguration.java | 443 ---
.../config/ServiceStreamConfigProvider.java | 88 -
.../service/config/StreamConfigProvider.java | 34 -
.../service/config/package-info.java | 21 -
.../distributedlog/service/package-info.java | 21 -
.../service/placement/EqualLoadAppraiser.java | 39 -
.../placement/LeastLoadPlacementPolicy.java | 200 -
.../service/placement/LoadAppraiser.java | 39 -
.../service/placement/PlacementPolicy.java | 148 -
.../placement/PlacementStateManager.java | 79 -
.../service/placement/ServerLoad.java | 158 -
.../service/placement/StreamLoad.java | 115 -
.../placement/ZKPlacementStateManager.java | 173 -
.../service/placement/package-info.java | 21 -
.../service/stream/AbstractStreamOp.java | 175 -
.../service/stream/AbstractWriteOp.java | 60 -
.../service/stream/BulkWriteOp.java | 253 --
.../distributedlog/service/stream/DeleteOp.java | 76 -
.../service/stream/HeartbeatOp.java | 101 -
.../service/stream/ReleaseOp.java | 76 -
.../distributedlog/service/stream/Stream.java | 93 -
.../service/stream/StreamFactory.java | 38 -
.../service/stream/StreamFactoryImpl.java | 95 -
.../service/stream/StreamImpl.java | 925 -----
.../service/stream/StreamManager.java | 142 -
.../service/stream/StreamManagerImpl.java | 413 --
.../distributedlog/service/stream/StreamOp.java | 75 -
.../service/stream/StreamOpStats.java | 104 -
.../service/stream/TruncateOp.java | 91 -
.../distributedlog/service/stream/WriteOp.java | 173 -
.../service/stream/WriteOpWithPayload.java | 27 -
.../service/stream/admin/AdminOp.java | 40 -
.../service/stream/admin/CreateOp.java | 57 -
.../service/stream/admin/StreamAdminOp.java | 100 -
.../service/stream/admin/package-info.java | 21 -
.../stream/limiter/DynamicRequestLimiter.java | 94 -
.../stream/limiter/RequestLimiterBuilder.java | 116 -
.../stream/limiter/ServiceRequestLimiter.java | 103 -
.../stream/limiter/StreamAcquireLimiter.java | 56 -
.../stream/limiter/StreamRequestLimiter.java | 87 -
.../service/stream/limiter/package-info.java | 21 -
.../service/stream/package-info.java | 21 -
.../CacheableStreamPartitionConverter.java | 58 -
.../DelimiterStreamPartitionConverter.java | 50 -
.../IdentityStreamPartitionConverter.java | 28 -
.../service/streamset/Partition.java | 99 -
.../service/streamset/PartitionMap.java | 58 -
.../streamset/StreamPartitionConverter.java | 35 -
.../service/streamset/package-info.java | 21 -
.../distributedlog/service/tools/ProxyTool.java | 350 --
.../service/tools/package-info.java | 21 -
.../service/utils/ServerUtils.java | 49 -
.../service/utils/package-info.java | 21 -
.../apache/bookkeeper/stats/package-info.java | 2 +-
.../distributedlog/service/ClientUtils.java | 33 +
.../service/DistributedLogCluster.java | 352 ++
.../service/DistributedLogServer.java | 460 +++
.../service/DistributedLogServerApp.java | 187 +
.../service/DistributedLogServiceImpl.java | 794 ++++
.../service/FatalErrorHandler.java | 30 +
.../distributedlog/service/MonitorService.java | 469 +++
.../service/MonitorServiceApp.java | 133 +
.../distributedlog/service/ResponseUtils.java | 86 +
.../service/ServerFeatureKeys.java | 29 +
.../distributedlog/service/StatsFilter.java | 60 +
.../service/announcer/Announcer.java | 41 +
.../service/announcer/NOPAnnouncer.java | 40 +
.../service/announcer/ServerSetAnnouncer.java | 111 +
.../service/announcer/package-info.java | 21 +
.../service/balancer/Balancer.java | 68 +
.../service/balancer/BalancerTool.java | 327 ++
.../service/balancer/BalancerUtils.java | 74 +
.../service/balancer/ClusterBalancer.java | 378 ++
.../balancer/CountBasedStreamChooser.java | 109 +
.../service/balancer/LimitedStreamChooser.java | 57 +
.../service/balancer/SimpleBalancer.java | 246 ++
.../service/balancer/StreamChooser.java | 30 +
.../service/balancer/StreamMover.java | 34 +
.../service/balancer/StreamMoverImpl.java | 94 +
.../service/balancer/package-info.java | 21 +
.../config/DefaultStreamConfigProvider.java | 73 +
.../config/NullStreamConfigProvider.java | 40 +
.../service/config/ServerConfiguration.java | 443 +++
.../config/ServiceStreamConfigProvider.java | 88 +
.../service/config/StreamConfigProvider.java | 34 +
.../service/config/package-info.java | 21 +
.../distributedlog/service/package-info.java | 21 +
.../service/placement/EqualLoadAppraiser.java | 39 +
.../placement/LeastLoadPlacementPolicy.java | 200 +
.../service/placement/LoadAppraiser.java | 39 +
.../service/placement/PlacementPolicy.java | 148 +
.../placement/PlacementStateManager.java | 79 +
.../service/placement/ServerLoad.java | 158 +
.../service/placement/StreamLoad.java | 115 +
.../placement/ZKPlacementStateManager.java | 173 +
.../service/placement/package-info.java | 21 +
.../service/stream/AbstractStreamOp.java | 175 +
.../service/stream/AbstractWriteOp.java | 60 +
.../service/stream/BulkWriteOp.java | 253 ++
.../distributedlog/service/stream/DeleteOp.java | 76 +
.../service/stream/HeartbeatOp.java | 101 +
.../service/stream/ReleaseOp.java | 76 +
.../distributedlog/service/stream/Stream.java | 93 +
.../service/stream/StreamFactory.java | 38 +
.../service/stream/StreamFactoryImpl.java | 95 +
.../service/stream/StreamImpl.java | 925 +++++
.../service/stream/StreamManager.java | 142 +
.../service/stream/StreamManagerImpl.java | 413 ++
.../distributedlog/service/stream/StreamOp.java | 75 +
.../service/stream/StreamOpStats.java | 104 +
.../service/stream/TruncateOp.java | 91 +
.../distributedlog/service/stream/WriteOp.java | 173 +
.../service/stream/WriteOpWithPayload.java | 27 +
.../service/stream/admin/AdminOp.java | 40 +
.../service/stream/admin/CreateOp.java | 57 +
.../service/stream/admin/StreamAdminOp.java | 100 +
.../service/stream/admin/package-info.java | 21 +
.../stream/limiter/DynamicRequestLimiter.java | 94 +
.../stream/limiter/RequestLimiterBuilder.java | 116 +
.../stream/limiter/ServiceRequestLimiter.java | 103 +
.../stream/limiter/StreamAcquireLimiter.java | 56 +
.../stream/limiter/StreamRequestLimiter.java | 87 +
.../service/stream/limiter/package-info.java | 21 +
.../service/stream/package-info.java | 21 +
.../CacheableStreamPartitionConverter.java | 58 +
.../DelimiterStreamPartitionConverter.java | 50 +
.../IdentityStreamPartitionConverter.java | 28 +
.../service/streamset/Partition.java | 99 +
.../service/streamset/PartitionMap.java | 58 +
.../streamset/StreamPartitionConverter.java | 35 +
.../service/streamset/package-info.java | 21 +
.../distributedlog/service/tools/ProxyTool.java | 350 ++
.../service/tools/package-info.java | 21 +
.../service/utils/ServerUtils.java | 49 +
.../service/utils/package-info.java | 21 +
.../src/main/resources/findbugsExclude.xml | 8 +-
.../src/main/thrift/metadata.thrift | 2 +-
.../client/routing/LocalRoutingService.java | 146 -
.../service/DistributedLogServerTestCase.java | 298 --
.../service/TestDistributedLogServerBase.java | 720 ----
.../TestDistributedLogServerClientRouting.java | 58 -
.../TestDistributedLogServerServerRouting.java | 28 -
.../service/TestDistributedLogService.java | 833 -----
.../service/TestRegionUnavailable.java | 140 -
.../distributedlog/service/TestStatsFilter.java | 58 -
.../service/balancer/TestBalancerUtils.java | 65 -
.../service/balancer/TestClusterBalancer.java | 189 -
.../balancer/TestCountBasedStreamChooser.java | 204 -
.../service/balancer/TestSimpleBalancer.java | 180 -
.../service/balancer/TestStreamMover.java | 86 -
.../service/config/TestServerConfiguration.java | 68 -
.../config/TestStreamConfigProvider.java | 140 -
.../placement/TestLeastLoadPlacementPolicy.java | 176 -
.../service/placement/TestServerLoad.java | 50 -
.../service/placement/TestStreamLoad.java | 37 -
.../placement/TestZKPlacementStateManager.java | 136 -
.../service/stream/TestStreamManager.java | 135 -
.../service/stream/TestStreamOp.java | 95 -
.../limiter/TestServiceRequestLimiter.java | 301 --
.../TestDelimiterStreamPartitionConverter.java | 52 -
.../TestIdentityStreamPartitionConverter.java | 43 -
.../service/streamset/TestPartitionMap.java | 68 -
.../service/utils/TestServerUtils.java | 41 -
.../client/routing/LocalRoutingService.java | 146 +
.../service/DistributedLogServerTestCase.java | 298 ++
.../service/TestDistributedLogServerBase.java | 720 ++++
.../TestDistributedLogServerClientRouting.java | 58 +
.../TestDistributedLogServerServerRouting.java | 28 +
.../service/TestDistributedLogService.java | 833 +++++
.../service/TestRegionUnavailable.java | 140 +
.../distributedlog/service/TestStatsFilter.java | 58 +
.../service/balancer/TestBalancerUtils.java | 65 +
.../service/balancer/TestClusterBalancer.java | 189 +
.../balancer/TestCountBasedStreamChooser.java | 204 +
.../service/balancer/TestSimpleBalancer.java | 180 +
.../service/balancer/TestStreamMover.java | 86 +
.../service/config/TestServerConfiguration.java | 68 +
.../config/TestStreamConfigProvider.java | 140 +
.../placement/TestLeastLoadPlacementPolicy.java | 176 +
.../service/placement/TestServerLoad.java | 50 +
.../service/placement/TestStreamLoad.java | 37 +
.../placement/TestZKPlacementStateManager.java | 136 +
.../service/stream/TestStreamManager.java | 135 +
.../service/stream/TestStreamOp.java | 95 +
.../limiter/TestServiceRequestLimiter.java | 301 ++
.../TestDelimiterStreamPartitionConverter.java | 52 +
.../TestIdentityStreamPartitionConverter.java | 43 +
.../service/streamset/TestPartitionMap.java | 68 +
.../service/utils/TestServerUtils.java | 41 +
.../distributedlog-basic/basic-1.md | 16 +-
.../distributedlog-basic/basic-2.md | 10 +-
.../distributedlog-basic/basic-3.md | 10 +-
.../distributedlog-basic/basic-4.md | 10 +-
.../distributedlog-basic/basic-5.md | 10 +-
.../distributedlog-basic/basic-6.md | 10 +-
.../distributedlog-basic/conf/log4j.properties | 8 +-
.../distributedlog/basic/AtomicWriter.java | 107 -
.../basic/ConsoleProxyMultiWriter.java | 91 -
.../basic/ConsoleProxyWriter.java | 77 -
.../distributedlog/basic/ConsoleWriter.java | 101 -
.../distributedlog/basic/MultiReader.java | 140 -
.../distributedlog/basic/RecordGenerator.java | 90 -
.../distributedlog/basic/StreamRewinder.java | 113 -
.../distributedlog/basic/TailReader.java | 112 -
.../distributedlog/basic/AtomicWriter.java | 107 +
.../basic/ConsoleProxyMultiWriter.java | 91 +
.../basic/ConsoleProxyWriter.java | 77 +
.../distributedlog/basic/ConsoleWriter.java | 101 +
.../distributedlog/basic/MultiReader.java | 140 +
.../distributedlog/basic/RecordGenerator.java | 90 +
.../distributedlog/basic/StreamRewinder.java | 113 +
.../apache/distributedlog/basic/TailReader.java | 112 +
.../distributedlog-kafka/conf/log4j.properties | 8 +-
.../kafka/DLFutureRecordMetadata.java | 104 -
.../kafka/KafkaDistributedLogProducer.java | 158 -
.../kafka/DLFutureRecordMetadata.java | 104 +
.../kafka/KafkaDistributedLogProducer.java | 158 +
.../distributedlog-mapreduce/README.md | 2 +-
.../mapreduce/DistributedLogInputFormat.java | 129 -
.../mapreduce/LogSegmentReader.java | 148 -
.../mapreduce/LogSegmentSplit.java | 95 -
.../mapreduce/DistributedLogInputFormat.java | 129 +
.../mapreduce/LogSegmentReader.java | 148 +
.../mapreduce/LogSegmentSplit.java | 95 +
.../conf/log4j.properties | 8 +-
.../distributedlog-messaging/messaging-1.md | 10 +-
.../distributedlog-messaging/messaging-2.md | 10 +-
.../ConsoleProxyPartitionedMultiWriter.java | 121 -
.../messaging/ConsoleProxyRRMultiWriter.java | 77 -
.../messaging/IdenticalTransformer.java | 25 -
.../messaging/IntPartitioner.java | 28 -
.../messaging/PartitionedMultiWriter.java | 50 -
.../distributedlog/messaging/Partitioner.java | 25 -
.../distributedlog/messaging/RRMultiWriter.java | 94 -
.../messaging/ReaderWithOffsets.java | 132 -
.../messaging/StreamTransformer.java | 188 -
.../distributedlog/messaging/Transformer.java | 34 -
.../ConsoleProxyPartitionedMultiWriter.java | 121 +
.../messaging/ConsoleProxyRRMultiWriter.java | 77 +
.../messaging/IdenticalTransformer.java | 25 +
.../messaging/IntPartitioner.java | 28 +
.../messaging/PartitionedMultiWriter.java | 50 +
.../distributedlog/messaging/Partitioner.java | 25 +
.../distributedlog/messaging/RRMultiWriter.java | 94 +
.../messaging/ReaderWithOffsets.java | 132 +
.../messaging/StreamTransformer.java | 188 +
.../distributedlog/messaging/Transformer.java | 34 +
.../src/main/resources/findbugsExclude.xml | 4 +-
.../src/main/thrift/messaging.thrift | 2 +-
docs/_plugins/jekyll-rst/.gitignore | 2 +-
docs/_plugins/jekyll-rst/LICENSE.txt | 2 +-
docs/_plugins/jekyll-rst/README.rst | 2 +-
docs/_plugins/jekyll-rst/converter.rb | 2 +-
docs/_plugins/jekyll-rst/directives.py | 2 +-
docs/_plugins/jekyll-rst/transform.py | 2 +-
docs/admin_guide/operations.rst | 6 +-
docs/deployment/cluster.rst | 6 +-
.../bootstrap/glyphicons-halflings-regular.svg | 2 +-
docs/js/bootstrap.min.js | 2 +-
docs/start/quickstart.rst | 4 +-
docs/tutorials/basic-1.rst | 16 +-
docs/tutorials/basic-2.rst | 10 +-
docs/tutorials/basic-3.rst | 10 +-
docs/tutorials/basic-4.rst | 10 +-
docs/tutorials/basic-5.rst | 10 +-
docs/tutorials/basic-6.rst | 10 +-
docs/tutorials/messaging-1.rst | 10 +-
docs/tutorials/messaging-2.rst | 10 +-
docs/user_guide/implementation/storage.rst | 2 +-
pom.xml | 3 +-
scripts/integration/smoketest.sh | 4 +-
.../bootstrap/glyphicons-halflings-regular.svg | 2 +-
website/js/bootstrap.min.js | 2 +-
1204 files changed, 100167 insertions(+), 100166 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/conf/log4j.properties b/distributedlog-benchmark/conf/log4j.properties
index 73b4cfa..930db8d 100644
--- a/distributedlog-benchmark/conf/log4j.properties
+++ b/distributedlog-benchmark/conf/log4j.properties
@@ -30,11 +30,11 @@ log4j.logger.org.apache.zookeeper=INFO
log4j.logger.org.apache.bookkeeper=INFO
# redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.com.twitter.distributedlog.util.MonitoredFuturePool=INFO, Executors
-log4j.logger.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
+log4j.logger.org.apache.distributedlog.util.MonitoredFuturePool=INFO, Executors
+log4j.logger.org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
log4j.logger.org.apache.bookkeeper.util.SafeRunnable=INFO, Executors
-log4j.additivity.com.twitter.distributedlog.util.MonitoredFuturePool=false
-log4j.additivity.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
+log4j.additivity.org.apache.distributedlog.util.MonitoredFuturePool=false
+log4j.additivity.org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
log4j.additivity.org.apache.bookkeeper.util.SafeRunnable=false
log4j.appender.Executors=org.apache.log4j.RollingFileAppender
[12/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
new file mode 100644
index 0000000..eedfbd6
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
@@ -0,0 +1,751 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import com.google.common.base.Ticker;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.IdleReaderException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.ReadCancelledException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.Utils;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import com.twitter.util.Throw;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Function1;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+/**
+ * BookKeeper based {@link AsyncLogReader} implementation.
+ *
+ * <h3>Metrics</h3>
+ * All the metrics are exposed under `async_reader`.
+ * <ul>
+ * <li> `async_reader`/future_set: opstats. time spent on satisfying futures of read requests.
+ * if it is high, it means that the caller takes time on processing the result of read requests.
+ * The side effect is blocking consequent reads.
+ * <li> `async_reader`/schedule: opstats. time spent on scheduling next reads.
+ * <li> `async_reader`/background_read: opstats. time spent on background reads.
+ * <li> `async_reader`/read_next_exec: opstats. time spent on executing {@link #readNext()}.
+ * <li> `async_reader`/time_between_read_next: opstats. time spent on between two consequent {@link #readNext()}.
+ * if it is high, it means that the caller is slowing down on calling {@link #readNext()}.
+ * <li> `async_reader`/delay_until_promise_satisfied: opstats. total latency for the read requests.
+ * <li> `async_reader`/idle_reader_error: counter. the number idle reader errors.
+ * </ul>
+ */
+class BKAsyncLogReader implements AsyncLogReader, Runnable, AsyncNotification {
+ static final Logger LOG = LoggerFactory.getLogger(BKAsyncLogReader.class);
+
+ private static final Function1<List<LogRecordWithDLSN>, LogRecordWithDLSN> READ_NEXT_MAP_FUNCTION =
+ new AbstractFunction1<List<LogRecordWithDLSN>, LogRecordWithDLSN>() {
+ @Override
+ public LogRecordWithDLSN apply(List<LogRecordWithDLSN> records) {
+ return records.get(0);
+ }
+ };
+
+ private final String streamName;
+ protected final BKDistributedLogManager bkDistributedLogManager;
+ protected final BKLogReadHandler readHandler;
+ private final AtomicReference<Throwable> lastException = new AtomicReference<Throwable>();
+ private final OrderedScheduler scheduler;
+ private final ConcurrentLinkedQueue<PendingReadRequest> pendingRequests = new ConcurrentLinkedQueue<PendingReadRequest>();
+ private final Object scheduleLock = new Object();
+ private final AtomicLong scheduleCount = new AtomicLong(0);
+ final private Stopwatch scheduleDelayStopwatch;
+ final private Stopwatch readNextDelayStopwatch;
+ private DLSN startDLSN;
+ private ReadAheadEntryReader readAheadReader = null;
+ private int lastPosition = 0;
+ private final boolean positionGapDetectionEnabled;
+ private final int idleErrorThresholdMillis;
+ final ScheduledFuture<?> idleReaderTimeoutTask;
+ private ScheduledFuture<?> backgroundScheduleTask = null;
+ // last process time
+ private final Stopwatch lastProcessTime;
+
+ protected Promise<Void> closeFuture = null;
+
+ private boolean lockStream = false;
+
+ private final boolean returnEndOfStreamRecord;
+
+ private final Runnable BACKGROUND_READ_SCHEDULER = new Runnable() {
+ @Override
+ public void run() {
+ synchronized (scheduleLock) {
+ backgroundScheduleTask = null;
+ }
+ scheduleBackgroundRead();
+ }
+ };
+
+ // State
+ private Entry.Reader currentEntry = null;
+ private LogRecordWithDLSN nextRecord = null;
+
+ // Failure Injector
+ private boolean disableProcessingReadRequests = false;
+
+ // Stats
+ private final OpStatsLogger readNextExecTime;
+ private final OpStatsLogger delayUntilPromiseSatisfied;
+ private final OpStatsLogger timeBetweenReadNexts;
+ private final OpStatsLogger futureSetLatency;
+ private final OpStatsLogger scheduleLatency;
+ private final OpStatsLogger backgroundReaderRunTime;
+ private final Counter idleReaderCheckCount;
+ private final Counter idleReaderCheckIdleReadRequestCount;
+ private final Counter idleReaderCheckIdleReadAheadCount;
+ private final Counter idleReaderError;
+
+ private class PendingReadRequest {
+ private final Stopwatch enqueueTime;
+ private final int numEntries;
+ private final List<LogRecordWithDLSN> records;
+ private final Promise<List<LogRecordWithDLSN>> promise;
+ private final long deadlineTime;
+ private final TimeUnit deadlineTimeUnit;
+
+ PendingReadRequest(int numEntries,
+ long deadlineTime,
+ TimeUnit deadlineTimeUnit) {
+ this.numEntries = numEntries;
+ this.enqueueTime = Stopwatch.createStarted();
+ // optimize the space usage for single read.
+ if (numEntries == 1) {
+ this.records = new ArrayList<LogRecordWithDLSN>(1);
+ } else {
+ this.records = new ArrayList<LogRecordWithDLSN>();
+ }
+ this.promise = new Promise<List<LogRecordWithDLSN>>();
+ this.deadlineTime = deadlineTime;
+ this.deadlineTimeUnit = deadlineTimeUnit;
+ }
+
+ Promise<List<LogRecordWithDLSN>> getPromise() {
+ return promise;
+ }
+
+ long elapsedSinceEnqueue(TimeUnit timeUnit) {
+ return enqueueTime.elapsed(timeUnit);
+ }
+
+ void setException(Throwable throwable) {
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ if (promise.updateIfEmpty(new Throw<List<LogRecordWithDLSN>>(throwable))) {
+ futureSetLatency.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ delayUntilPromiseSatisfied.registerFailedEvent(enqueueTime.elapsed(TimeUnit.MICROSECONDS));
+ }
+ }
+
+ boolean hasReadRecords() {
+ return records.size() > 0;
+ }
+
+ boolean hasReadEnoughRecords() {
+ return records.size() >= numEntries;
+ }
+
+ long getRemainingWaitTime() {
+ if (deadlineTime <= 0L) {
+ return 0L;
+ }
+ return deadlineTime - elapsedSinceEnqueue(deadlineTimeUnit);
+ }
+
+ void addRecord(LogRecordWithDLSN record) {
+ records.add(record);
+ }
+
+ void complete() {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("{} : Satisfied promise with {} records", readHandler.getFullyQualifiedName(), records.size());
+ }
+ delayUntilPromiseSatisfied.registerSuccessfulEvent(enqueueTime.stop().elapsed(TimeUnit.MICROSECONDS));
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ promise.setValue(records);
+ futureSetLatency.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ }
+ }
+
+ BKAsyncLogReader(BKDistributedLogManager bkdlm,
+ OrderedScheduler scheduler,
+ DLSN startDLSN,
+ Optional<String> subscriberId,
+ boolean returnEndOfStreamRecord,
+ StatsLogger statsLogger) {
+ this.streamName = bkdlm.getStreamName();
+ this.bkDistributedLogManager = bkdlm;
+ this.scheduler = scheduler;
+ this.readHandler = bkDistributedLogManager.createReadHandler(subscriberId,
+ this, true);
+ LOG.debug("Starting async reader at {}", startDLSN);
+ this.startDLSN = startDLSN;
+ this.scheduleDelayStopwatch = Stopwatch.createUnstarted();
+ this.readNextDelayStopwatch = Stopwatch.createStarted();
+ this.positionGapDetectionEnabled = bkdlm.getConf().getPositionGapDetectionEnabled();
+ this.idleErrorThresholdMillis = bkdlm.getConf().getReaderIdleErrorThresholdMillis();
+ this.returnEndOfStreamRecord = returnEndOfStreamRecord;
+
+ // Stats
+ StatsLogger asyncReaderStatsLogger = statsLogger.scope("async_reader");
+ futureSetLatency = asyncReaderStatsLogger.getOpStatsLogger("future_set");
+ scheduleLatency = asyncReaderStatsLogger.getOpStatsLogger("schedule");
+ backgroundReaderRunTime = asyncReaderStatsLogger.getOpStatsLogger("background_read");
+ readNextExecTime = asyncReaderStatsLogger.getOpStatsLogger("read_next_exec");
+ timeBetweenReadNexts = asyncReaderStatsLogger.getOpStatsLogger("time_between_read_next");
+ delayUntilPromiseSatisfied = asyncReaderStatsLogger.getOpStatsLogger("delay_until_promise_satisfied");
+ idleReaderError = asyncReaderStatsLogger.getCounter("idle_reader_error");
+ idleReaderCheckCount = asyncReaderStatsLogger.getCounter("idle_reader_check_total");
+ idleReaderCheckIdleReadRequestCount = asyncReaderStatsLogger.getCounter("idle_reader_check_idle_read_requests");
+ idleReaderCheckIdleReadAheadCount = asyncReaderStatsLogger.getCounter("idle_reader_check_idle_readahead");
+
+ // Lock the stream if requested. The lock will be released when the reader is closed.
+ this.lockStream = false;
+ this.idleReaderTimeoutTask = scheduleIdleReaderTaskIfNecessary();
+ this.lastProcessTime = Stopwatch.createStarted();
+ }
+
+ private ScheduledFuture<?> scheduleIdleReaderTaskIfNecessary() {
+ if (idleErrorThresholdMillis < Integer.MAX_VALUE) {
+ // Dont run the task more than once every seconds (for sanity)
+ long period = Math.max(idleErrorThresholdMillis / 10, 1000);
+ // Except when idle reader threshold is less than a second (tests?)
+ period = Math.min(period, idleErrorThresholdMillis / 5);
+
+ return scheduler.scheduleAtFixedRate(streamName, new Runnable() {
+ @Override
+ public void run() {
+ PendingReadRequest nextRequest = pendingRequests.peek();
+
+ idleReaderCheckCount.inc();
+ if (null == nextRequest) {
+ return;
+ }
+
+ idleReaderCheckIdleReadRequestCount.inc();
+ if (nextRequest.elapsedSinceEnqueue(TimeUnit.MILLISECONDS) < idleErrorThresholdMillis) {
+ return;
+ }
+
+ ReadAheadEntryReader readAheadReader = getReadAheadReader();
+
+ // read request has been idle
+ // - cache has records but read request are idle,
+ // that means notification was missed between readahead and reader.
+ // - cache is empty and readahead is idle (no records added for a long time)
+ idleReaderCheckIdleReadAheadCount.inc();
+ try {
+ if (null == readAheadReader || (!hasMoreRecords() &&
+ readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS))) {
+ markReaderAsIdle();
+ return;
+ } else if (lastProcessTime.elapsed(TimeUnit.MILLISECONDS) > idleErrorThresholdMillis) {
+ markReaderAsIdle();;
+ }
+ } catch (IOException e) {
+ setLastException(e);
+ return;
+ }
+ }
+ }, period, period, TimeUnit.MILLISECONDS);
+ }
+ return null;
+ }
+
+ synchronized ReadAheadEntryReader getReadAheadReader() {
+ return readAheadReader;
+ }
+
+ void cancelIdleReaderTask() {
+ // Do this after we have checked that the reader was not previously closed
+ try {
+ if (null != idleReaderTimeoutTask) {
+ idleReaderTimeoutTask.cancel(true);
+ }
+ } catch (Exception exc) {
+ LOG.info("{}: Failed to cancel the background idle reader timeout task", readHandler.getFullyQualifiedName());
+ }
+ }
+
+ private void markReaderAsIdle() {
+ idleReaderError.inc();
+ IdleReaderException ire = new IdleReaderException("Reader on stream "
+ + readHandler.getFullyQualifiedName()
+ + " is idle for " + idleErrorThresholdMillis +" ms");
+ setLastException(ire);
+ // cancel all pending reads directly rather than notifying on error
+ // because idle reader could happen on idle read requests that usually means something wrong
+ // in scheduling reads
+ cancelAllPendingReads(ire);
+ }
+
+ protected synchronized void setStartDLSN(DLSN fromDLSN) throws UnexpectedException {
+ if (null != readAheadReader) {
+ throw new UnexpectedException("Could't reset from dlsn after reader already starts reading.");
+ }
+ startDLSN = fromDLSN;
+ }
+
+ @VisibleForTesting
+ public synchronized DLSN getStartDLSN() {
+ return startDLSN;
+ }
+
+ public Future<Void> lockStream() {
+ this.lockStream = true;
+ return readHandler.lockStream();
+ }
+
+ private boolean checkClosedOrInError(String operation) {
+ if (null == lastException.get()) {
+ try {
+ if (null != readHandler && null != getReadAheadReader()) {
+ getReadAheadReader().checkLastException();
+ }
+
+ bkDistributedLogManager.checkClosedOrInError(operation);
+ } catch (IOException exc) {
+ setLastException(exc);
+ }
+ }
+
+ if (lockStream) {
+ try {
+ readHandler.checkReadLock();
+ } catch (IOException ex) {
+ setLastException(ex);
+ }
+ }
+
+ if (null != lastException.get()) {
+ LOG.trace("Cancelling pending reads");
+ cancelAllPendingReads(lastException.get());
+ return true;
+ }
+
+ return false;
+ }
+
+ private void setLastException(IOException exc) {
+ lastException.compareAndSet(null, exc);
+ }
+
+ @Override
+ public String getStreamName() {
+ return streamName;
+ }
+
+ /**
+ * @return A promise that when satisfied will contain the Log Record with its DLSN.
+ */
+ @Override
+ public synchronized Future<LogRecordWithDLSN> readNext() {
+ return readInternal(1, 0, TimeUnit.MILLISECONDS).map(READ_NEXT_MAP_FUNCTION);
+ }
+
+ public synchronized Future<List<LogRecordWithDLSN>> readBulk(int numEntries) {
+ return readInternal(numEntries, 0, TimeUnit.MILLISECONDS);
+ }
+
+ @Override
+ public synchronized Future<List<LogRecordWithDLSN>> readBulk(int numEntries,
+ long waitTime,
+ TimeUnit timeUnit) {
+ return readInternal(numEntries, waitTime, timeUnit);
+ }
+
+ /**
+ * Read up to <i>numEntries</i> entries. The future will be satisfied when any number of entries are
+ * ready (1 to <i>numEntries</i>).
+ *
+ * @param numEntries
+ * num entries to read
+ * @return A promise that satisfied with a non-empty list of log records with their DLSN.
+ */
+ private synchronized Future<List<LogRecordWithDLSN>> readInternal(int numEntries,
+ long deadlineTime,
+ TimeUnit deadlineTimeUnit) {
+ timeBetweenReadNexts.registerSuccessfulEvent(readNextDelayStopwatch.elapsed(TimeUnit.MICROSECONDS));
+ readNextDelayStopwatch.reset().start();
+ final PendingReadRequest readRequest = new PendingReadRequest(numEntries, deadlineTime, deadlineTimeUnit);
+
+ if (null == readAheadReader) {
+ final ReadAheadEntryReader readAheadEntryReader = this.readAheadReader = new ReadAheadEntryReader(
+ getStreamName(),
+ getStartDLSN(),
+ bkDistributedLogManager.getConf(),
+ readHandler,
+ bkDistributedLogManager.getReaderEntryStore(),
+ bkDistributedLogManager.getScheduler(),
+ Ticker.systemTicker(),
+ bkDistributedLogManager.alertStatsLogger);
+ readHandler.checkLogStreamExists().addEventListener(new FutureEventListener<Void>() {
+ @Override
+ public void onSuccess(Void value) {
+ try {
+ readHandler.registerListener(readAheadEntryReader);
+ readHandler.asyncStartFetchLogSegments()
+ .map(new AbstractFunction1<Versioned<List<LogSegmentMetadata>>, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(Versioned<List<LogSegmentMetadata>> logSegments) {
+ readAheadEntryReader.addStateChangeNotification(BKAsyncLogReader.this);
+ readAheadEntryReader.start(logSegments.getValue());
+ return BoxedUnit.UNIT;
+ }
+ });
+ } catch (Exception exc) {
+ notifyOnError(exc);
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ notifyOnError(cause);
+ }
+ });
+ }
+
+ if (checkClosedOrInError("readNext")) {
+ readRequest.setException(lastException.get());
+ } else {
+ boolean queueEmpty = pendingRequests.isEmpty();
+ pendingRequests.add(readRequest);
+
+ if (queueEmpty) {
+ scheduleBackgroundRead();
+ }
+ }
+
+ readNextExecTime.registerSuccessfulEvent(readNextDelayStopwatch.elapsed(TimeUnit.MICROSECONDS));
+ readNextDelayStopwatch.reset().start();
+
+ return readRequest.getPromise();
+ }
+
+ public synchronized void scheduleBackgroundRead() {
+ // if the reader is already closed, we don't need to schedule background read again.
+ if (null != closeFuture) {
+ return;
+ }
+
+ long prevCount = scheduleCount.getAndIncrement();
+ if (0 == prevCount) {
+ scheduleDelayStopwatch.reset().start();
+ scheduler.submit(streamName, this);
+ }
+ }
+
+ @Override
+ public Future<Void> asyncClose() {
+ // Cancel the idle reader timeout task, interrupting if necessary
+ ReadCancelledException exception;
+ Promise<Void> closePromise;
+ synchronized (this) {
+ if (null != closeFuture) {
+ return closeFuture;
+ }
+ closePromise = closeFuture = new Promise<Void>();
+ exception = new ReadCancelledException(readHandler.getFullyQualifiedName(), "Reader was closed");
+ setLastException(exception);
+ }
+
+ // Do this after we have checked that the reader was not previously closed
+ cancelIdleReaderTask();
+
+ synchronized (scheduleLock) {
+ if (null != backgroundScheduleTask) {
+ backgroundScheduleTask.cancel(true);
+ }
+ }
+
+ cancelAllPendingReads(exception);
+
+ ReadAheadEntryReader readAheadReader = getReadAheadReader();
+ if (null != readAheadReader) {
+ readHandler.unregisterListener(readAheadReader);
+ readAheadReader.removeStateChangeNotification(this);
+ }
+ Utils.closeSequence(bkDistributedLogManager.getScheduler(), true,
+ readAheadReader,
+ readHandler
+ ).proxyTo(closePromise);
+ return closePromise;
+ }
+
+ private void cancelAllPendingReads(Throwable throwExc) {
+ for (PendingReadRequest promise : pendingRequests) {
+ promise.setException(throwExc);
+ }
+ pendingRequests.clear();
+ }
+
+ synchronized boolean hasMoreRecords() throws IOException {
+ if (null == readAheadReader) {
+ return false;
+ }
+ if (readAheadReader.getNumCachedEntries() > 0 || null != nextRecord) {
+ return true;
+ } else if (null != currentEntry) {
+ nextRecord = currentEntry.nextRecord();
+ return null != nextRecord;
+ }
+ return false;
+ }
+
+ private synchronized LogRecordWithDLSN readNextRecord() throws IOException {
+ if (null == readAheadReader) {
+ return null;
+ }
+ if (null == currentEntry) {
+ currentEntry = readAheadReader.getNextReadAheadEntry(0L, TimeUnit.MILLISECONDS);
+ // no entry after reading from read ahead then return null
+ if (null == currentEntry) {
+ return null;
+ }
+ }
+
+ LogRecordWithDLSN recordToReturn;
+ if (null == nextRecord) {
+ nextRecord = currentEntry.nextRecord();
+ // no more records in current entry
+ if (null == nextRecord) {
+ currentEntry = null;
+ return readNextRecord();
+ }
+ }
+
+ // found a record to return and prefetch the next one
+ recordToReturn = nextRecord;
+ nextRecord = currentEntry.nextRecord();
+ return recordToReturn;
+ }
+
+ @Override
+ public void run() {
+ synchronized(scheduleLock) {
+ if (scheduleDelayStopwatch.isRunning()) {
+ scheduleLatency.registerSuccessfulEvent(scheduleDelayStopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ }
+
+ Stopwatch runTime = Stopwatch.createStarted();
+ int iterations = 0;
+ long scheduleCountLocal = scheduleCount.get();
+ LOG.debug("{}: Scheduled Background Reader", readHandler.getFullyQualifiedName());
+ while(true) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("{}: Executing Iteration: {}", readHandler.getFullyQualifiedName(), iterations++);
+ }
+
+ PendingReadRequest nextRequest = null;
+ synchronized(this) {
+ nextRequest = pendingRequests.peek();
+
+ // Queue is empty, nothing to read, return
+ if (null == nextRequest) {
+ LOG.trace("{}: Queue Empty waiting for Input", readHandler.getFullyQualifiedName());
+ scheduleCount.set(0);
+ backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
+ return;
+ }
+
+ if (disableProcessingReadRequests) {
+ LOG.info("Reader of {} is forced to stop processing read requests", readHandler.getFullyQualifiedName());
+ return;
+ }
+ }
+ 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
+ if (null == lastException.get()) {
+ if (nextRequest.getPromise().isInterrupted().isDefined()) {
+ setLastException(new DLInterruptedException("Interrupted on reading " + readHandler.getFullyQualifiedName() + " : ",
+ nextRequest.getPromise().isInterrupted().get()));
+ }
+ }
+
+ if (checkClosedOrInError("readNext")) {
+ if (!(lastException.get().getCause() instanceof LogNotFoundException)) {
+ LOG.warn("{}: Exception", readHandler.getFullyQualifiedName(), lastException.get());
+ }
+ backgroundReaderRunTime.registerFailedEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
+ return;
+ }
+
+ try {
+ // Fail 10% of the requests when asked to simulate errors
+ if (bkDistributedLogManager.getFailureInjector().shouldInjectErrors()) {
+ throw new IOException("Reader Simulated Exception");
+ }
+ LogRecordWithDLSN record;
+ while (!nextRequest.hasReadEnoughRecords()) {
+ // read single record
+ do {
+ record = readNextRecord();
+ } while (null != record && (record.isControl() || (record.getDlsn().compareTo(getStartDLSN()) < 0)));
+ if (null == record) {
+ break;
+ } else {
+ if (record.isEndOfStream() && !returnEndOfStreamRecord) {
+ setLastException(new EndOfStreamException("End of Stream Reached for "
+ + readHandler.getFullyQualifiedName()));
+ break;
+ }
+
+ // gap detection
+ if (recordPositionsContainsGap(record, lastPosition)) {
+ bkDistributedLogManager.raiseAlert("Gap detected between records at record = {}", record);
+ if (positionGapDetectionEnabled) {
+ throw new DLIllegalStateException("Gap detected between records at record = " + record);
+ }
+ }
+ lastPosition = record.getLastPositionWithinLogSegment();
+
+ nextRequest.addRecord(record);
+ }
+ };
+ } catch (IOException exc) {
+ setLastException(exc);
+ if (!(exc instanceof LogNotFoundException)) {
+ LOG.warn("{} : read with skip Exception", readHandler.getFullyQualifiedName(), lastException.get());
+ }
+ continue;
+ }
+
+ if (nextRequest.hasReadRecords()) {
+ long remainingWaitTime = nextRequest.getRemainingWaitTime();
+ if (remainingWaitTime > 0 && !nextRequest.hasReadEnoughRecords()) {
+ backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
+ scheduleDelayStopwatch.reset().start();
+ scheduleCount.set(0);
+ // the request could still wait for more records
+ backgroundScheduleTask = scheduler.schedule(
+ streamName,
+ BACKGROUND_READ_SCHEDULER,
+ remainingWaitTime,
+ nextRequest.deadlineTimeUnit);
+ return;
+ }
+
+ PendingReadRequest request = pendingRequests.poll();
+ if (null != request && nextRequest == request) {
+ request.complete();
+ if (null != backgroundScheduleTask) {
+ backgroundScheduleTask.cancel(true);
+ backgroundScheduleTask = null;
+ }
+ } else {
+ DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at dlsn = "
+ + nextRequest.records.get(0).getDlsn());
+ nextRequest.setException(ise);
+ if (null != request) {
+ request.setException(ise);
+ }
+ // We should never get here as we should have exited the loop if
+ // pendingRequests were empty
+ bkDistributedLogManager.raiseAlert("Unexpected condition at dlsn = {}",
+ nextRequest.records.get(0).getDlsn());
+ setLastException(ise);
+ }
+ } else {
+ if (0 == scheduleCountLocal) {
+ LOG.trace("Schedule count dropping to zero", lastException.get());
+ backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
+ return;
+ }
+ scheduleCountLocal = scheduleCount.decrementAndGet();
+ }
+ }
+ }
+ }
+
+ private boolean recordPositionsContainsGap(LogRecordWithDLSN record, long lastPosition) {
+ final boolean firstLogRecord = (1 == record.getPositionWithinLogSegment());
+ final boolean endOfStreamRecord = record.isEndOfStream();
+ final boolean emptyLogSegment = (0 == lastPosition);
+ final boolean positionIncreasedByOne = (record.getPositionWithinLogSegment() == (lastPosition + 1));
+
+ return !firstLogRecord && !endOfStreamRecord && !emptyLogSegment &&
+ !positionIncreasedByOne;
+ }
+
+ /**
+ * Triggered when the background activity encounters an exception
+ */
+ @Override
+ public void notifyOnError(Throwable cause) {
+ if (cause instanceof IOException) {
+ setLastException((IOException) cause);
+ } else {
+ setLastException(new IOException(cause));
+ }
+ scheduleBackgroundRead();
+ }
+
+ /**
+ * Triggered when the background activity completes an operation
+ */
+ @Override
+ public void notifyOnOperationComplete() {
+ scheduleBackgroundRead();
+ }
+
+ @VisibleForTesting
+ void simulateErrors() {
+ bkDistributedLogManager.getFailureInjector().injectErrors(true);
+ }
+
+ @VisibleForTesting
+ synchronized void disableReadAheadLogSegmentsNotification() {
+ readHandler.disableReadAheadLogSegmentsNotification();
+ }
+
+ @VisibleForTesting
+ synchronized void disableProcessingReadRequests() {
+ disableProcessingReadRequests = true;
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
new file mode 100644
index 0000000..1102ff5
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
@@ -0,0 +1,559 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.StreamNotReadyException;
+import org.apache.distributedlog.exceptions.WriteCancelledException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.feature.CoreFeatureKeys;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.util.FutureUtils;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import com.twitter.util.Try;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Function1;
+import scala.Option;
+import scala.runtime.AbstractFunction1;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * BookKeeper based {@link AsyncLogWriter} implementation.
+ *
+ * <h3>Metrics</h3>
+ * All the metrics are exposed under `log_writer`.
+ * <ul>
+ * <li> `log_writer/write`: opstats. latency characteristics about the time that write operations spent.
+ * <li> `log_writer/bulk_write`: opstats. latency characteristics about the time that bulk_write
+ * operations spent.
+ * are pending in the queue for long time due to log segment rolling.
+ * <li> `log_writer/get_writer`: opstats. the time spent on getting the writer. it could spike when there
+ * is log segment rolling happened during getting the writer. it is a good stat to look into when the latency
+ * is caused by queuing time.
+ * <li> `log_writer/pending_request_dispatch`: counter. the number of queued operations that are dispatched
+ * after log segment is rolled. it is an metric on measuring how many operations has been queued because of
+ * log segment rolling.
+ * </ul>
+ * See {@link BKLogSegmentWriter} for segment writer stats.
+ */
+public class BKAsyncLogWriter extends BKAbstractLogWriter implements AsyncLogWriter {
+
+ static final Logger LOG = LoggerFactory.getLogger(BKAsyncLogWriter.class);
+
+ static Function1<List<LogSegmentMetadata>, Boolean> TruncationResultConverter =
+ new AbstractFunction1<List<LogSegmentMetadata>, Boolean>() {
+ @Override
+ public Boolean apply(List<LogSegmentMetadata> segments) {
+ return true;
+ }
+ };
+
+ // Records pending for roll log segment.
+ class PendingLogRecord implements FutureEventListener<DLSN> {
+
+ final LogRecord record;
+ final Promise<DLSN> promise;
+ final boolean flush;
+
+ PendingLogRecord(LogRecord record, boolean flush) {
+ this.record = record;
+ this.promise = new Promise<DLSN>();
+ this.flush = flush;
+ }
+
+ @Override
+ public void onSuccess(DLSN value) {
+ promise.setValue(value);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ promise.setException(cause);
+ encounteredError = true;
+ }
+ }
+
+ /**
+ * Last pending record in current log segment. After it is satisified, it would
+ * roll log segment.
+ *
+ * This implementation is based on the assumption that all future satisified in same
+ * order future pool.
+ */
+ class LastPendingLogRecord extends PendingLogRecord {
+
+ LastPendingLogRecord(LogRecord record, boolean flush) {
+ super(record, flush);
+ }
+
+ @Override
+ public void onSuccess(DLSN value) {
+ super.onSuccess(value);
+ // roll log segment and issue all pending requests.
+ rollLogSegmentAndIssuePendingRequests(record.getTransactionId());
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ super.onFailure(cause);
+ // error out pending requests.
+ errorOutPendingRequestsAndWriter(cause);
+ }
+ }
+
+ private final boolean streamFailFast;
+ private final boolean disableRollOnSegmentError;
+ private LinkedList<PendingLogRecord> pendingRequests = null;
+ private volatile boolean encounteredError = false;
+ private Promise<BKLogSegmentWriter> rollingFuture = null;
+ private long lastTxId = DistributedLogConstants.INVALID_TXID;
+
+ private final StatsLogger statsLogger;
+ private final OpStatsLogger writeOpStatsLogger;
+ private final OpStatsLogger markEndOfStreamOpStatsLogger;
+ private final OpStatsLogger bulkWriteOpStatsLogger;
+ private final OpStatsLogger getWriterOpStatsLogger;
+ private final Counter pendingRequestDispatch;
+
+ private final Feature disableLogSegmentRollingFeature;
+
+ BKAsyncLogWriter(DistributedLogConfiguration conf,
+ DynamicDistributedLogConfiguration dynConf,
+ BKDistributedLogManager bkdlm,
+ BKLogWriteHandler writeHandler, /** log writer owns the handler **/
+ FeatureProvider featureProvider,
+ StatsLogger dlmStatsLogger) {
+ super(conf, dynConf, bkdlm);
+ this.writeHandler = writeHandler;
+ this.streamFailFast = conf.getFailFastOnStreamNotReady();
+ this.disableRollOnSegmentError = conf.getDisableRollingOnLogSegmentError();
+
+ // features
+ disableLogSegmentRollingFeature = featureProvider.getFeature(CoreFeatureKeys.DISABLE_LOGSEGMENT_ROLLING.name().toLowerCase());
+ // stats
+ this.statsLogger = dlmStatsLogger.scope("log_writer");
+ this.writeOpStatsLogger = statsLogger.getOpStatsLogger("write");
+ this.markEndOfStreamOpStatsLogger = statsLogger.getOpStatsLogger("mark_end_of_stream");
+ this.bulkWriteOpStatsLogger = statsLogger.getOpStatsLogger("bulk_write");
+ this.getWriterOpStatsLogger = statsLogger.getOpStatsLogger("get_writer");
+ this.pendingRequestDispatch = statsLogger.getCounter("pending_request_dispatch");
+ }
+
+ @VisibleForTesting
+ synchronized void setLastTxId(long txId) {
+ lastTxId = Math.max(lastTxId, txId);
+ }
+
+ @Override
+ public synchronized long getLastTxId() {
+ return lastTxId;
+ }
+
+ /**
+ * Write a log record as control record. The method will be used by Monitor Service to enforce a new inprogress segment.
+ *
+ * @param record
+ * log record
+ * @return future of the write
+ */
+ public Future<DLSN> writeControlRecord(final LogRecord record) {
+ record.setControl();
+ return write(record);
+ }
+
+ private BKLogSegmentWriter getCachedLogSegmentWriter() throws WriteException {
+ if (encounteredError) {
+ throw new WriteException(bkDistributedLogManager.getStreamName(),
+ "writer has been closed due to error.");
+ }
+ BKLogSegmentWriter segmentWriter = getCachedLogWriter();
+ if (null != segmentWriter
+ && segmentWriter.isLogSegmentInError()
+ && !disableRollOnSegmentError) {
+ return null;
+ } else {
+ return segmentWriter;
+ }
+ }
+
+ private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
+ boolean bestEffort,
+ boolean rollLog,
+ boolean allowMaxTxID) {
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ return FutureUtils.stats(
+ doGetLogSegmentWriter(firstTxid, bestEffort, rollLog, allowMaxTxID),
+ getWriterOpStatsLogger,
+ stopwatch);
+ }
+
+ private Future<BKLogSegmentWriter> doGetLogSegmentWriter(final long firstTxid,
+ final boolean bestEffort,
+ final boolean rollLog,
+ final boolean allowMaxTxID) {
+ if (encounteredError) {
+ return Future.exception(new WriteException(bkDistributedLogManager.getStreamName(),
+ "writer has been closed due to error."));
+ }
+ Future<BKLogSegmentWriter> writerFuture = asyncGetLedgerWriter(!disableRollOnSegmentError);
+ if (null == writerFuture) {
+ return rollLogSegmentIfNecessary(null, firstTxid, bestEffort, allowMaxTxID);
+ } else if (rollLog) {
+ return writerFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<BKLogSegmentWriter>>() {
+ @Override
+ public Future<BKLogSegmentWriter> apply(BKLogSegmentWriter writer) {
+ return rollLogSegmentIfNecessary(writer, firstTxid, bestEffort, allowMaxTxID);
+ }
+ });
+ } else {
+ return writerFuture;
+ }
+ }
+
+ /**
+ * We write end of stream marker by writing a record with MAX_TXID, so we need to allow using
+ * max txid when rolling for this case only.
+ */
+ private Future<BKLogSegmentWriter> getLogSegmentWriterForEndOfStream() {
+ return getLogSegmentWriter(DistributedLogConstants.MAX_TXID,
+ false /* bestEffort */,
+ false /* roll log */,
+ true /* allow max txid */);
+ }
+
+ private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
+ boolean bestEffort,
+ boolean rollLog) {
+ return getLogSegmentWriter(firstTxid, bestEffort, rollLog, false /* allow max txid */);
+ }
+
+ Future<DLSN> queueRequest(LogRecord record, boolean flush) {
+ PendingLogRecord pendingLogRecord = new PendingLogRecord(record, flush);
+ pendingRequests.add(pendingLogRecord);
+ return pendingLogRecord.promise;
+ }
+
+ boolean shouldRollLog(BKLogSegmentWriter w) {
+ try {
+ return null == w ||
+ (!disableLogSegmentRollingFeature.isAvailable() &&
+ shouldStartNewSegment(w));
+ } catch (IOException ioe) {
+ return false;
+ }
+ }
+
+ void startQueueingRequests() {
+ assert(null == pendingRequests && null == rollingFuture);
+ pendingRequests = new LinkedList<PendingLogRecord>();
+ rollingFuture = new Promise<BKLogSegmentWriter>();
+ }
+
+ // for ordering guarantee, we shouldn't send requests to next log segments until
+ // previous log segment is done.
+ private synchronized Future<DLSN> asyncWrite(final LogRecord record,
+ boolean flush) {
+ // The passed in writer may be stale since we acquire the writer outside of sync
+ // lock. If we recently rolled and the new writer is cached, use that instead.
+ Future<DLSN> result = null;
+ BKLogSegmentWriter w;
+ try {
+ w = getCachedLogSegmentWriter();
+ } catch (WriteException we) {
+ return Future.exception(we);
+ }
+ if (null != rollingFuture) {
+ if (streamFailFast) {
+ result = Future.exception(new StreamNotReadyException("Rolling log segment"));
+ } else {
+ result = queueRequest(record, flush);
+ }
+ } else if (shouldRollLog(w)) {
+ // insert a last record, so when it called back, we will trigger a log segment rolling
+ startQueueingRequests();
+ if (null != w) {
+ LastPendingLogRecord lastLogRecordInCurrentSegment = new LastPendingLogRecord(record, flush);
+ w.asyncWrite(record, true).addEventListener(lastLogRecordInCurrentSegment);
+ result = lastLogRecordInCurrentSegment.promise;
+ } else { // no log segment yet. roll the log segment and issue pending requests.
+ result = queueRequest(record, flush);
+ rollLogSegmentAndIssuePendingRequests(record.getTransactionId());
+ }
+ } else {
+ result = w.asyncWrite(record, flush);
+ }
+ // use map here rather than onSuccess because we want lastTxId to be updated before
+ // satisfying the future
+ return result.map(new AbstractFunction1<DLSN, DLSN>() {
+ @Override
+ public DLSN apply(DLSN dlsn) {
+ setLastTxId(record.getTransactionId());
+ return dlsn;
+ }
+ });
+ }
+
+ private List<Future<DLSN>> asyncWriteBulk(List<LogRecord> records) {
+ final ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(records.size());
+ Iterator<LogRecord> iterator = records.iterator();
+ while (iterator.hasNext()) {
+ LogRecord record = iterator.next();
+ Future<DLSN> future = asyncWrite(record, !iterator.hasNext());
+ results.add(future);
+
+ // Abort early if an individual write has already failed.
+ Option<Try<DLSN>> result = future.poll();
+ if (result.isDefined() && result.get().isThrow()) {
+ break;
+ }
+ }
+ if (records.size() > results.size()) {
+ appendCancelledFutures(results, records.size() - results.size());
+ }
+ return results;
+ }
+
+ private void appendCancelledFutures(List<Future<DLSN>> futures, int numToAdd) {
+ final WriteCancelledException cre =
+ new WriteCancelledException(getStreamName());
+ for (int i = 0; i < numToAdd; i++) {
+ Future<DLSN> cancelledFuture = Future.exception(cre);
+ futures.add(cancelledFuture);
+ }
+ }
+
+ private void rollLogSegmentAndIssuePendingRequests(final long firstTxId) {
+ getLogSegmentWriter(firstTxId, true, true)
+ .addEventListener(new FutureEventListener<BKLogSegmentWriter>() {
+ @Override
+ public void onSuccess(BKLogSegmentWriter writer) {
+ try {
+ synchronized (BKAsyncLogWriter.this) {
+ for (PendingLogRecord pendingLogRecord : pendingRequests) {
+ FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_LogWriterIssuePending);
+ writer.asyncWrite(pendingLogRecord.record, pendingLogRecord.flush)
+ .addEventListener(pendingLogRecord);
+ }
+ // if there are no records in the pending queue, let's write a control record
+ // so that when a new log segment is rolled, a control record will be added and
+ // the corresponding bookies would be able to create its ledger.
+ if (pendingRequests.isEmpty()) {
+ LogRecord controlRecord = new LogRecord(firstTxId,
+ DistributedLogConstants.CONTROL_RECORD_CONTENT);
+ controlRecord.setControl();
+ PendingLogRecord controlReq = new PendingLogRecord(controlRecord, false);
+ writer.asyncWrite(controlReq.record, controlReq.flush)
+ .addEventListener(controlReq);
+ }
+ if (null != rollingFuture) {
+ FutureUtils.setValue(rollingFuture, writer);
+ }
+ rollingFuture = null;
+ pendingRequestDispatch.add(pendingRequests.size());
+ pendingRequests = null;
+ }
+ } catch (IOException ioe) {
+ errorOutPendingRequestsAndWriter(ioe);
+ }
+ }
+ @Override
+ public void onFailure(Throwable cause) {
+ errorOutPendingRequestsAndWriter(cause);
+ }
+ });
+ }
+
+ @VisibleForTesting
+ void errorOutPendingRequests(Throwable cause, boolean errorOutWriter) {
+ final List<PendingLogRecord> pendingRequestsSnapshot;
+ synchronized (this) {
+ pendingRequestsSnapshot = pendingRequests;
+ encounteredError = errorOutWriter;
+ pendingRequests = null;
+ if (null != rollingFuture) {
+ FutureUtils.setException(rollingFuture, cause);
+ }
+ rollingFuture = null;
+ }
+
+ pendingRequestDispatch.add(pendingRequestsSnapshot.size());
+
+ // After erroring out the writer above, no more requests
+ // will be enqueued to pendingRequests
+ for (PendingLogRecord pendingLogRecord : pendingRequestsSnapshot) {
+ pendingLogRecord.promise.setException(cause);
+ }
+ }
+
+ void errorOutPendingRequestsAndWriter(Throwable cause) {
+ errorOutPendingRequests(cause, true /* error out writer */);
+ }
+
+ /**
+ * Write a log record to the stream.
+ *
+ * @param record single log record
+ */
+ @Override
+ public Future<DLSN> write(final LogRecord record) {
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ return FutureUtils.stats(
+ asyncWrite(record, true),
+ writeOpStatsLogger,
+ stopwatch);
+ }
+
+ /**
+ * Write many log records to the stream. The return type here is unfortunate but its a direct result
+ * of having to combine FuturePool and the asyncWriteBulk method which returns a future as well. The
+ * problem is the List that asyncWriteBulk returns can't be materialized until getLogSegmentWriter
+ * completes, so it has to be wrapped in a future itself.
+ *
+ * @param records list of records
+ */
+ @Override
+ public Future<List<Future<DLSN>>> writeBulk(final List<LogRecord> records) {
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ return FutureUtils.stats(
+ Future.value(asyncWriteBulk(records)),
+ bulkWriteOpStatsLogger,
+ stopwatch);
+ }
+
+ @Override
+ public Future<Boolean> truncate(final DLSN dlsn) {
+ if (DLSN.InvalidDLSN == dlsn) {
+ return Future.value(false);
+ }
+ BKLogWriteHandler writeHandler;
+ try {
+ writeHandler = getWriteHandler();
+ } catch (IOException e) {
+ return Future.exception(e);
+ }
+ return writeHandler.setLogSegmentsOlderThanDLSNTruncated(dlsn).map(TruncationResultConverter);
+ }
+
+ Future<Long> flushAndCommit() {
+ Future<BKLogSegmentWriter> writerFuture;
+ synchronized (this) {
+ if (null != this.rollingFuture) {
+ writerFuture = this.rollingFuture;
+ } else {
+ writerFuture = getCachedLogWriterFuture();
+ }
+ }
+ if (null == writerFuture) {
+ return Future.value(getLastTxId());
+ }
+ return writerFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
+ @Override
+ public Future<Long> apply(BKLogSegmentWriter writer) {
+ return writer.flushAndCommit();
+ }
+ });
+ }
+
+ Future<Long> markEndOfStream() {
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ Future<BKLogSegmentWriter> logSegmentWriterFuture;
+ synchronized (this) {
+ logSegmentWriterFuture = this.rollingFuture;
+ }
+ if (null == logSegmentWriterFuture) {
+ logSegmentWriterFuture = getLogSegmentWriterForEndOfStream();
+ }
+
+ return FutureUtils.stats(
+ logSegmentWriterFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
+ @Override
+ public Future<Long> apply(BKLogSegmentWriter w) {
+ return w.markEndOfStream();
+ }
+ }),
+ markEndOfStreamOpStatsLogger,
+ stopwatch);
+ }
+
+ @Override
+ protected Future<Void> asyncCloseAndComplete() {
+ Future<BKLogSegmentWriter> logSegmentWriterFuture;
+ synchronized (this) {
+ logSegmentWriterFuture = this.rollingFuture;
+ }
+
+ if (null == logSegmentWriterFuture) {
+ return super.asyncCloseAndComplete();
+ } else {
+ return logSegmentWriterFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Void>>() {
+ @Override
+ public Future<Void> apply(BKLogSegmentWriter segmentWriter) {
+ return BKAsyncLogWriter.super.asyncCloseAndComplete();
+ }
+ });
+ }
+ }
+
+ @Override
+ void closeAndComplete() throws IOException {
+ FutureUtils.result(asyncCloseAndComplete());
+ }
+
+ /**
+ * *TEMP HACK*
+ * Get the name of the stream this writer writes data to
+ */
+ @Override
+ public String getStreamName() {
+ return bkDistributedLogManager.getStreamName();
+ }
+
+ @Override
+ public Future<Void> asyncAbort() {
+ Future<Void> result = super.asyncAbort();
+ synchronized (this) {
+ if (pendingRequests != null) {
+ for (PendingLogRecord pendingLogRecord : pendingRequests) {
+ pendingLogRecord.promise.setException(new WriteException(bkDistributedLogManager.getStreamName(),
+ "abort wring: writer has been closed due to error."));
+ }
+ }
+ }
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("AsyncLogWriter:%s", getStreamName());
+ }
+}
[11/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
new file mode 100644
index 0000000..00e6b5c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
@@ -0,0 +1,1106 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.LogEmptyException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.function.CloseAsyncCloseableFunction;
+import org.apache.distributedlog.function.GetVersionedValueFunction;
+import org.apache.distributedlog.injector.AsyncFailureInjector;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
+import org.apache.distributedlog.metadata.LogMetadataForReader;
+import org.apache.distributedlog.metadata.LogMetadataForWriter;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.lock.NopDistributedLock;
+import org.apache.distributedlog.lock.ZKDistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentFilter;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
+import org.apache.distributedlog.metadata.LogStreamMetadataStore;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.stats.BroadCastStatsLogger;
+import org.apache.distributedlog.subscription.SubscriptionsStore;
+import org.apache.distributedlog.util.Allocator;
+import org.apache.distributedlog.util.DLUtils;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.MonitoredFuturePool;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.PermitLimiter;
+import org.apache.distributedlog.util.PermitManager;
+import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.distributedlog.util.Utils;
+import com.twitter.util.ExceptionalFunction;
+import com.twitter.util.ExceptionalFunction0;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+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.StatsLogger;
+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;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.distributedlog.namespace.NamespaceDriver.Role.READER;
+import static org.apache.distributedlog.namespace.NamespaceDriver.Role.WRITER;
+
+/**
+ * <h3>Metrics</h3>
+ * <ul>
+ * <li> `log_writer/*`: all asynchronous writer related metrics are exposed under scope `log_writer`.
+ * See {@link BKAsyncLogWriter} for detail stats.
+ * <li> `async_reader/*`: all asyncrhonous reader related metrics are exposed under scope `async_reader`.
+ * See {@link BKAsyncLogReader} for detail stats.
+ * <li> `writer_future_pool/*`: metrics about the future pools that used by writers are exposed under
+ * scope `writer_future_pool`. See {@link MonitoredFuturePool} for detail stats.
+ * <li> `reader_future_pool/*`: metrics about the future pools that used by readers are exposed under
+ * scope `reader_future_pool`. See {@link MonitoredFuturePool} for detail stats.
+ * <li> `lock/*`: metrics about the locks used by writers. See {@link ZKDistributedLock} for detail
+ * stats.
+ * <li> `read_lock/*`: metrics about the locks used by readers. See {@link ZKDistributedLock} for
+ * detail stats.
+ * <li> `logsegments/*`: metrics about basic operations on log segments. See {@link BKLogHandler} for details.
+ * <li> `segments/*`: metrics about write operations on log segments. See {@link BKLogWriteHandler} for details.
+ * <li> `readahead_worker/*`: metrics about readahead workers used by readers. See {@link BKLogReadHandler}
+ * for details.
+ * </ul>
+ */
+class BKDistributedLogManager implements DistributedLogManager {
+ static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogManager.class);
+
+ static final Function<LogRecordWithDLSN, Long> RECORD_2_TXID_FUNCTION =
+ new Function<LogRecordWithDLSN, Long>() {
+ @Override
+ public Long apply(LogRecordWithDLSN record) {
+ return record.getTransactionId();
+ }
+ };
+
+ static final Function<LogRecordWithDLSN, DLSN> RECORD_2_DLSN_FUNCTION =
+ new Function<LogRecordWithDLSN, DLSN>() {
+ @Override
+ public DLSN apply(LogRecordWithDLSN record) {
+ return record.getDlsn();
+ }
+ };
+
+ 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 FeatureProvider featureProvider;
+ private final AsyncFailureInjector failureInjector;
+ private final StatsLogger statsLogger;
+ private final StatsLogger perLogStatsLogger;
+ final AlertStatsLogger alertStatsLogger;
+
+ // log segment metadata cache
+ private final LogSegmentMetadataCache logSegmentMetadataCache;
+
+ //
+ // Writer Related Variables
+ //
+ private final PermitLimiter writeLimiter;
+
+ //
+ // Reader Related Variables
+ ///
+ // read handler for listener.
+ private BKLogReadHandler readHandlerForListener = null;
+ private final PendingReaders pendingReaders;
+
+ // 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 driver namespace driver
+ * @param logSegmentMetadataCache log segment metadata cache
+ * @param scheduler ordered scheduled used by readers and writers
+ * @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 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
+ */
+ BKDistributedLogManager(String name,
+ DistributedLogConfiguration conf,
+ DynamicDistributedLogConfiguration dynConf,
+ URI uri,
+ NamespaceDriver driver,
+ LogSegmentMetadataCache logSegmentMetadataCache,
+ OrderedScheduler scheduler,
+ String clientId,
+ Integer regionId,
+ PermitLimiter writeLimiter,
+ FeatureProvider featureProvider,
+ AsyncFailureInjector failureInjector,
+ StatsLogger 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.statsLogger = statsLogger;
+ this.perLogStatsLogger = BroadCastStatsLogger.masterslave(perLogStatsLogger, statsLogger);
+ this.pendingReaders = new PendingReaders(scheduler);
+ this.regionId = regionId;
+ this.clientId = clientId;
+ this.streamIdentifier = conf.getUnpartitionedStreamName();
+ this.writeLimiter = writeLimiter;
+ // Feature Provider
+ this.featureProvider = featureProvider;
+ // Failure Injector
+ this.failureInjector = failureInjector;
+ // Stats
+ this.alertStatsLogger = new AlertStatsLogger(this.perLogStatsLogger, "dl_alert");
+ this.resourcesCloseable = resourcesCloseable;
+ }
+
+ @Override
+ public String getStreamName() {
+ return name;
+ }
+
+ @Override
+ public NamespaceDriver getNamespaceDriver() {
+ return driver;
+ }
+
+ URI getUri() {
+ return uri;
+ }
+
+ DistributedLogConfiguration getConf() {
+ return conf;
+ }
+
+ OrderedScheduler getScheduler() {
+ return scheduler;
+ }
+
+ AsyncFailureInjector getFailureInjector() {
+ return failureInjector;
+ }
+
+ //
+ // Test Methods
+ //
+
+ @VisibleForTesting
+ LogStreamMetadataStore getWriterMetadataStore() {
+ return driver.getLogStreamMetadataStore(WRITER);
+ }
+
+ @VisibleForTesting
+ LogSegmentEntryStore getReaderEntryStore() {
+ return driver.getLogSegmentEntryStore(READER);
+ }
+
+ @VisibleForTesting
+ FeatureProvider getFeatureProvider() {
+ return this.featureProvider;
+ }
+
+ private synchronized BKLogReadHandler getReadHandlerAndRegisterListener(
+ boolean create, LogSegmentListener listener) {
+ if (null == readHandlerForListener && create) {
+ readHandlerForListener = createReadHandler();
+ readHandlerForListener.registerListener(listener);
+ // start fetch the log segments after created the listener
+ readHandlerForListener.asyncStartFetchLogSegments();
+ return readHandlerForListener;
+ }
+ if (null != readHandlerForListener && null != listener) {
+ readHandlerForListener.registerListener(listener);
+ }
+ return readHandlerForListener;
+ }
+
+ @Override
+ public List<LogSegmentMetadata> getLogSegments() throws IOException {
+ return FutureUtils.result(getLogSegmentsAsync());
+ }
+
+ protected Future<List<LogSegmentMetadata>> getLogSegmentsAsync() {
+ final BKLogReadHandler readHandler = createReadHandler();
+ return readHandler.readLogSegmentsFromStore(
+ LogSegmentMetadata.COMPARATOR,
+ LogSegmentFilter.DEFAULT_FILTER,
+ null)
+ .map(GetVersionedValueFunction.GET_LOGSEGMENT_LIST_FUNC)
+ .ensure(CloseAsyncCloseableFunction.of(readHandler));
+ }
+
+ @Override
+ public void registerListener(LogSegmentListener listener) throws IOException {
+ getReadHandlerAndRegisterListener(true, listener);
+ }
+
+ @Override
+ public synchronized void unregisterListener(LogSegmentListener listener) {
+ if (null != readHandlerForListener) {
+ readHandlerForListener.unregisterListener(listener);
+ }
+ }
+
+ public void checkClosedOrInError(String operation) throws AlreadyClosedException {
+ synchronized (this) {
+ if (null != closePromise) {
+ throw new AlreadyClosedException("Executing " + operation + " on already closed DistributedLogManager");
+ }
+ }
+ }
+
+ // Create Read Handler
+
+ synchronized BKLogReadHandler createReadHandler() {
+ Optional<String> subscriberId = Optional.absent();
+ return createReadHandler(subscriberId, false);
+ }
+
+ synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId) {
+ return createReadHandler(subscriberId, false);
+ }
+
+ synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId,
+ boolean isHandleForReading) {
+ return createReadHandler(
+ subscriberId,
+ null,
+ isHandleForReading);
+ }
+
+ synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId,
+ AsyncNotification notification,
+ boolean isHandleForReading) {
+ LogMetadataForReader logMetadata = LogMetadataForReader.of(uri, name, streamIdentifier);
+ return new BKLogReadHandler(
+ logMetadata,
+ subscriberId,
+ conf,
+ dynConf,
+ driver.getLogStreamMetadataStore(READER),
+ logSegmentMetadataCache,
+ driver.getLogSegmentEntryStore(READER),
+ scheduler,
+ alertStatsLogger,
+ statsLogger,
+ perLogStatsLogger,
+ clientId,
+ notification,
+ isHandleForReading);
+ }
+
+ // Create Ledger Allocator
+
+
+
+ // Create Write Handler
+
+ public BKLogWriteHandler createWriteHandler(boolean lockHandler)
+ throws IOException {
+ return FutureUtils.result(asyncCreateWriteHandler(lockHandler));
+ }
+
+ Future<BKLogWriteHandler> asyncCreateWriteHandler(final boolean lockHandler) {
+ // Fetching Log Metadata (create if not exists)
+ return driver.getLogStreamMetadataStore(WRITER).getLog(
+ uri,
+ name,
+ true,
+ conf.getCreateStreamIfNotExists()
+ ).flatMap(new AbstractFunction1<LogMetadataForWriter, Future<BKLogWriteHandler>>() {
+ @Override
+ public Future<BKLogWriteHandler> apply(LogMetadataForWriter logMetadata) {
+ Promise<BKLogWriteHandler> createPromise = new Promise<BKLogWriteHandler>();
+ createWriteHandler(logMetadata, lockHandler, createPromise);
+ return createPromise;
+ }
+ });
+ }
+
+ private void createWriteHandler(LogMetadataForWriter logMetadata,
+ boolean lockHandler,
+ final Promise<BKLogWriteHandler> createPromise) {
+ // Build the locks
+ DistributedLock lock;
+ if (conf.isWriteLockEnabled()) {
+ lock = driver.getLogStreamMetadataStore(WRITER).createWriteLock(logMetadata);
+ } else {
+ lock = NopDistributedLock.INSTANCE;
+ }
+
+ Allocator<LogSegmentEntryWriter, Object> segmentAllocator;
+ try {
+ segmentAllocator = driver.getLogSegmentEntryStore(WRITER)
+ .newLogSegmentAllocator(logMetadata, dynConf);
+ } catch (IOException ioe) {
+ FutureUtils.setException(createPromise, ioe);
+ return;
+ }
+
+ // Make sure writer handler created before resources are initialized
+ final BKLogWriteHandler writeHandler = new BKLogWriteHandler(
+ logMetadata,
+ conf,
+ driver.getLogStreamMetadataStore(WRITER),
+ logSegmentMetadataCache,
+ driver.getLogSegmentEntryStore(WRITER),
+ scheduler,
+ segmentAllocator,
+ statsLogger,
+ perLogStatsLogger,
+ alertStatsLogger,
+ clientId,
+ regionId,
+ writeLimiter,
+ featureProvider,
+ dynConf,
+ lock);
+ if (lockHandler) {
+ writeHandler.lockHandler().addEventListener(new FutureEventListener<DistributedLock>() {
+ @Override
+ public void onSuccess(DistributedLock lock) {
+ FutureUtils.setValue(createPromise, writeHandler);
+ }
+
+ @Override
+ public void onFailure(final Throwable cause) {
+ writeHandler.asyncClose().ensure(new AbstractFunction0<BoxedUnit>() {
+ @Override
+ public BoxedUnit apply() {
+ FutureUtils.setException(createPromise, cause);
+ return BoxedUnit.UNIT;
+ }
+ });
+ }
+ });
+ } else {
+ FutureUtils.setValue(createPromise, writeHandler);
+ }
+ }
+
+ PermitManager getLogSegmentRollingPermitManager() {
+ return driver.getLogStreamMetadataStore(WRITER).getPermitManager();
+ }
+
+ <T> Future<T> processReaderOperation(final Function<BKLogReadHandler, Future<T>> func) {
+ return scheduler.apply(new ExceptionalFunction0<BKLogReadHandler>() {
+ @Override
+ public BKLogReadHandler applyE() throws Throwable {
+ return getReadHandlerAndRegisterListener(true, null);
+ }
+ }).flatMap(new ExceptionalFunction<BKLogReadHandler, Future<T>>() {
+ @Override
+ public Future<T> applyE(final BKLogReadHandler readHandler) throws Throwable {
+ return func.apply(readHandler);
+ }
+ });
+ }
+
+ /**
+ * Check if an end of stream marker was added to the stream
+ * A stream with an end of stream marker cannot be appended to
+ *
+ * @return true if the marker was added to the stream, false otherwise
+ */
+ @Override
+ public boolean isEndOfStreamMarked() throws IOException {
+ checkClosedOrInError("isEndOfStreamMarked");
+ long lastTxId = FutureUtils.result(getLastLogRecordAsyncInternal(false, true)).getTransactionId();
+ return lastTxId == DistributedLogConstants.MAX_TXID;
+ }
+
+ /**
+ * Begin appending to the end of the log stream which is being treated as a sequence of bytes
+ *
+ * @return the writer interface to generate log records
+ */
+ public AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException {
+ long position;
+ try {
+ position = FutureUtils.result(getLastLogRecordAsyncInternal(true, false)).getTransactionId();
+ if (DistributedLogConstants.INVALID_TXID == position ||
+ DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID == position) {
+ position = 0;
+ }
+ } catch (LogEmptyException ex) {
+ position = 0;
+ } catch (LogNotFoundException ex) {
+ position = 0;
+ }
+ return new AppendOnlyStreamWriter(startAsyncLogSegmentNonPartitioned(), position);
+ }
+
+ /**
+ * Get a reader to read a log stream as a sequence of bytes
+ *
+ * @return the writer interface to generate log records
+ */
+ public AppendOnlyStreamReader getAppendOnlyStreamReader() throws IOException {
+ return new AppendOnlyStreamReader(this);
+ }
+
+ /**
+ * Begin writing to the log stream identified by the name
+ *
+ * @return the writer interface to generate log records
+ */
+ @Override
+ public BKSyncLogWriter startLogSegmentNonPartitioned() throws IOException {
+ checkClosedOrInError("startLogSegmentNonPartitioned");
+ BKSyncLogWriter writer = new BKSyncLogWriter(conf, dynConf, this);
+ boolean success = false;
+ try {
+ writer.createAndCacheWriteHandler();
+ BKLogWriteHandler writeHandler = writer.getWriteHandler();
+ FutureUtils.result(writeHandler.lockHandler());
+ success = true;
+ return writer;
+ } finally {
+ if (!success) {
+ writer.abort();
+ }
+ }
+ }
+
+ /**
+ * Begin writing to the log stream identified by the name
+ *
+ * @return the writer interface to generate log records
+ */
+ @Override
+ public BKAsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException {
+ return (BKAsyncLogWriter) FutureUtils.result(openAsyncLogWriter());
+ }
+
+ @Override
+ public Future<AsyncLogWriter> openAsyncLogWriter() {
+ try {
+ checkClosedOrInError("startLogSegmentNonPartitioned");
+ } catch (AlreadyClosedException e) {
+ return Future.exception(e);
+ }
+
+ Future<BKLogWriteHandler> createWriteHandleFuture;
+ synchronized (this) {
+ // 1. create the locked write handler
+ createWriteHandleFuture = asyncCreateWriteHandler(true);
+ }
+ return createWriteHandleFuture.flatMap(new AbstractFunction1<BKLogWriteHandler, Future<AsyncLogWriter>>() {
+ @Override
+ public Future<AsyncLogWriter> apply(final BKLogWriteHandler writeHandler) {
+ final BKAsyncLogWriter writer;
+ synchronized (BKDistributedLogManager.this) {
+ // 2. create the writer with the handler
+ writer = new BKAsyncLogWriter(
+ conf,
+ dynConf,
+ BKDistributedLogManager.this,
+ writeHandler,
+ featureProvider,
+ statsLogger);
+ }
+ // 3. recover the incomplete log segments
+ return writeHandler.recoverIncompleteLogSegments()
+ .map(new AbstractFunction1<Long, AsyncLogWriter>() {
+ @Override
+ public AsyncLogWriter apply(Long lastTxId) {
+ // 4. update last tx id if successfully recovered
+ writer.setLastTxId(lastTxId);
+ return writer;
+ }
+ }).onFailure(new AbstractFunction1<Throwable, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(Throwable cause) {
+ // 5. close the writer if recovery failed
+ writer.asyncAbort();
+ return BoxedUnit.UNIT;
+ }
+ });
+ }
+ });
+ }
+
+ @Override
+ public Future<DLSN> getDLSNNotLessThanTxId(final long fromTxnId) {
+ return getLogSegmentsAsync().flatMap(new AbstractFunction1<List<LogSegmentMetadata>, Future<DLSN>>() {
+ @Override
+ public Future<DLSN> apply(List<LogSegmentMetadata> segments) {
+ return getDLSNNotLessThanTxId(fromTxnId, segments);
+ }
+ });
+ }
+
+ private Future<DLSN> getDLSNNotLessThanTxId(long fromTxnId,
+ final List<LogSegmentMetadata> segments) {
+ if (segments.isEmpty()) {
+ return getLastDLSNAsync();
+ }
+ final int segmentIdx = DLUtils.findLogSegmentNotLessThanTxnId(segments, fromTxnId);
+ if (segmentIdx < 0) {
+ return Future.value(new DLSN(segments.get(0).getLogSegmentSequenceNumber(), 0L, 0L));
+ }
+ return getDLSNNotLessThanTxIdInSegment(
+ fromTxnId,
+ segmentIdx,
+ segments,
+ driver.getLogSegmentEntryStore(READER)
+ );
+ }
+
+ private Future<DLSN> getDLSNNotLessThanTxIdInSegment(final long fromTxnId,
+ final int segmentIdx,
+ final List<LogSegmentMetadata> segments,
+ final LogSegmentEntryStore entryStore) {
+ final LogSegmentMetadata segment = segments.get(segmentIdx);
+ return ReadUtils.getLogRecordNotLessThanTxId(
+ name,
+ segment,
+ fromTxnId,
+ scheduler,
+ entryStore,
+ Math.max(2, dynConf.getReadAheadBatchSize())
+ ).flatMap(new AbstractFunction1<Optional<LogRecordWithDLSN>, Future<DLSN>>() {
+ @Override
+ public Future<DLSN> apply(Optional<LogRecordWithDLSN> foundRecord) {
+ if (foundRecord.isPresent()) {
+ return Future.value(foundRecord.get().getDlsn());
+ }
+ if ((segments.size() - 1) == segmentIdx) {
+ return getLastLogRecordAsync().map(new AbstractFunction1<LogRecordWithDLSN, DLSN>() {
+ @Override
+ public DLSN apply(LogRecordWithDLSN record) {
+ if (record.getTransactionId() >= fromTxnId) {
+ return record.getDlsn();
+ }
+ return record.getDlsn().getNextDLSN();
+ }
+ });
+ } else {
+ return getDLSNNotLessThanTxIdInSegment(
+ fromTxnId,
+ segmentIdx + 1,
+ segments,
+ entryStore);
+ }
+ }
+ });
+ }
+
+ /**
+ * Get the input stream starting with fromTxnId for the specified log
+ *
+ * @param fromTxnId - the first transaction id we want to read
+ * @return the stream starting with transaction fromTxnId
+ * @throws IOException if a stream cannot be found.
+ */
+ @Override
+ public LogReader getInputStream(long fromTxnId)
+ throws IOException {
+ return getInputStreamInternal(fromTxnId);
+ }
+
+ @Override
+ public LogReader getInputStream(DLSN fromDLSN) throws IOException {
+ return getInputStreamInternal(fromDLSN, Optional.<Long>absent());
+ }
+
+ @Override
+ public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException {
+ return FutureUtils.result(openAsyncLogReader(fromTxnId));
+ }
+
+ /**
+ * Opening a log reader positioning by transaction id <code>fromTxnId</code>.
+ *
+ * <p>
+ * - retrieve log segments for the stream
+ * - if the log segment list is empty, positioning by the last dlsn
+ * - otherwise, find the first log segment that contains the records whose transaction ids are not less than
+ * the provided transaction id <code>fromTxnId</code>
+ * - if all log segments' records' transaction ids are more than <code>fromTxnId</code>, positioning
+ * on the first record.
+ * - otherwise, search the log segment to find the log record
+ * - if the log record is found, positioning the reader by that found record's dlsn
+ * - otherwise, positioning by the last dlsn
+ * </p>
+ *
+ * @see DLUtils#findLogSegmentNotLessThanTxnId(List, long)
+ * @see ReadUtils#getLogRecordNotLessThanTxId(String, LogSegmentMetadata, long, ExecutorService, LogSegmentEntryStore, int)
+ * @param fromTxnId
+ * transaction id to start reading from
+ * @return future representing the open result.
+ */
+ @Override
+ public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId) {
+ final Promise<DLSN> dlsnPromise = new Promise<DLSN>();
+ getDLSNNotLessThanTxId(fromTxnId).addEventListener(new FutureEventListener<DLSN>() {
+
+ @Override
+ public void onSuccess(DLSN dlsn) {
+ dlsnPromise.setValue(dlsn);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ if (cause instanceof LogEmptyException) {
+ dlsnPromise.setValue(DLSN.InitialDLSN);
+ } else {
+ dlsnPromise.setException(cause);
+ }
+ }
+ });
+ return dlsnPromise.flatMap(new AbstractFunction1<DLSN, Future<AsyncLogReader>>() {
+ @Override
+ public Future<AsyncLogReader> apply(DLSN dlsn) {
+ return openAsyncLogReader(dlsn);
+ }
+ });
+ }
+
+ @Override
+ public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException {
+ return FutureUtils.result(openAsyncLogReader(fromDLSN));
+ }
+
+ @Override
+ public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN) {
+ Optional<String> subscriberId = Optional.absent();
+ AsyncLogReader reader = new BKAsyncLogReader(
+ this,
+ scheduler,
+ fromDLSN,
+ subscriberId,
+ false,
+ statsLogger);
+ pendingReaders.add(reader);
+ return Future.value(reader);
+ }
+
+ /**
+ * Note the lock here is a sort of elective exclusive lock. I.e. acquiring this lock will only prevent other
+ * people who try to acquire the lock from reading from the stream. Normal readers (and writers) will not be
+ * blocked.
+ */
+ @Override
+ public Future<AsyncLogReader> getAsyncLogReaderWithLock(final DLSN fromDLSN) {
+ Optional<String> subscriberId = Optional.absent();
+ return getAsyncLogReaderWithLock(Optional.of(fromDLSN), subscriberId);
+ }
+
+ @Override
+ public Future<AsyncLogReader> getAsyncLogReaderWithLock(final DLSN fromDLSN, final String subscriberId) {
+ return getAsyncLogReaderWithLock(Optional.of(fromDLSN), Optional.of(subscriberId));
+ }
+
+ @Override
+ public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId) {
+ Optional<DLSN> fromDLSN = Optional.absent();
+ return getAsyncLogReaderWithLock(fromDLSN, Optional.of(subscriberId));
+ }
+
+ protected Future<AsyncLogReader> getAsyncLogReaderWithLock(final Optional<DLSN> fromDLSN,
+ final Optional<String> subscriberId) {
+ if (!fromDLSN.isPresent() && !subscriberId.isPresent()) {
+ return Future.exception(new UnexpectedException("Neither from dlsn nor subscriber id is provided."));
+ }
+ final BKAsyncLogReader reader = new BKAsyncLogReader(
+ BKDistributedLogManager.this,
+ scheduler,
+ fromDLSN.isPresent() ? fromDLSN.get() : DLSN.InitialDLSN,
+ subscriberId,
+ false,
+ statsLogger);
+ pendingReaders.add(reader);
+ final Future<Void> lockFuture = reader.lockStream();
+ final Promise<AsyncLogReader> createPromise = new Promise<AsyncLogReader>(
+ new Function<Throwable, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(Throwable cause) {
+ // cancel the lock when the creation future is cancelled
+ lockFuture.cancel();
+ return BoxedUnit.UNIT;
+ }
+ });
+ // lock the stream - fetch the last commit position on success
+ lockFuture.flatMap(new Function<Void, Future<AsyncLogReader>>() {
+ @Override
+ public Future<AsyncLogReader> apply(Void complete) {
+ if (fromDLSN.isPresent()) {
+ return Future.value((AsyncLogReader) reader);
+ }
+ LOG.info("Reader {} @ {} reading last commit position from subscription store after acquired lock.",
+ subscriberId.get(), name);
+ // we acquired lock
+ 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 {}.",
+ new Object[] { subscriberId.get(), name, lastCommitPosition });
+ reader.setStartDLSN(lastCommitPosition);
+ return reader;
+ }
+ });
+ }
+ }).addEventListener(new FutureEventListener<AsyncLogReader>() {
+ @Override
+ public void onSuccess(AsyncLogReader r) {
+ pendingReaders.remove(reader);
+ FutureUtils.setValue(createPromise, r);
+ }
+
+ @Override
+ public void onFailure(final Throwable cause) {
+ pendingReaders.remove(reader);
+ reader.asyncClose().ensure(new AbstractFunction0<BoxedUnit>() {
+ @Override
+ public BoxedUnit apply() {
+ FutureUtils.setException(createPromise, cause);
+ return BoxedUnit.UNIT;
+ }
+ });
+ }
+ });
+ return createPromise;
+ }
+
+ /**
+ * Get the input stream starting with fromTxnId for the specified log
+ *
+ * @param fromTxnId
+ * transaction id to start reading from
+ * @return log reader
+ * @throws IOException
+ */
+ LogReader getInputStreamInternal(long fromTxnId)
+ throws IOException {
+ DLSN fromDLSN;
+ try {
+ fromDLSN = FutureUtils.result(getDLSNNotLessThanTxId(fromTxnId));
+ } catch (LogEmptyException lee) {
+ fromDLSN = DLSN.InitialDLSN;
+ }
+ return getInputStreamInternal(fromDLSN, Optional.of(fromTxnId));
+ }
+
+ LogReader getInputStreamInternal(DLSN fromDLSN, Optional<Long> fromTxnId)
+ throws IOException {
+ LOG.info("Create sync reader starting from {}", fromDLSN);
+ checkClosedOrInError("getInputStream");
+ return new BKSyncLogReader(
+ conf,
+ this,
+ fromDLSN,
+ fromTxnId,
+ statsLogger);
+ }
+
+ /**
+ * Get the last log record in the stream
+ *
+ * @return the last log record in the stream
+ * @throws java.io.IOException if a stream cannot be found.
+ */
+ @Override
+ public LogRecordWithDLSN getLastLogRecord() throws IOException {
+ checkClosedOrInError("getLastLogRecord");
+ return FutureUtils.result(getLastLogRecordAsync());
+ }
+
+ @Override
+ public long getFirstTxId() throws IOException {
+ checkClosedOrInError("getFirstTxId");
+ return FutureUtils.result(getFirstRecordAsyncInternal()).getTransactionId();
+ }
+
+ @Override
+ public long getLastTxId() throws IOException {
+ checkClosedOrInError("getLastTxId");
+ return FutureUtils.result(getLastTxIdAsync());
+ }
+
+ @Override
+ public DLSN getLastDLSN() throws IOException {
+ checkClosedOrInError("getLastDLSN");
+ return FutureUtils.result(getLastLogRecordAsyncInternal(false, false)).getDlsn();
+ }
+
+ /**
+ * Get Latest log record in the log
+ *
+ * @return latest log record
+ */
+ @Override
+ public Future<LogRecordWithDLSN> getLastLogRecordAsync() {
+ return getLastLogRecordAsyncInternal(false, false);
+ }
+
+ private Future<LogRecordWithDLSN> getLastLogRecordAsyncInternal(final boolean recover,
+ final boolean includeEndOfStream) {
+ return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
+ @Override
+ public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
+ return ledgerHandler.getLastLogRecordAsync(recover, includeEndOfStream);
+ }
+ });
+ }
+
+ /**
+ * Get Latest Transaction Id in the log
+ *
+ * @return latest transaction id
+ */
+ @Override
+ public Future<Long> getLastTxIdAsync() {
+ return getLastLogRecordAsyncInternal(false, false)
+ .map(RECORD_2_TXID_FUNCTION);
+ }
+
+ /**
+ * Get first DLSN in the log.
+ *
+ * @return first dlsn in the stream
+ */
+ @Override
+ public Future<DLSN> getFirstDLSNAsync() {
+ return getFirstRecordAsyncInternal().map(RECORD_2_DLSN_FUNCTION);
+ }
+
+ private Future<LogRecordWithDLSN> getFirstRecordAsyncInternal() {
+ return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
+ @Override
+ public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
+ return ledgerHandler.asyncGetFirstLogRecord();
+ }
+ });
+ }
+
+ /**
+ * Get Latest DLSN in the log.
+ *
+ * @return latest transaction id
+ */
+ @Override
+ public Future<DLSN> getLastDLSNAsync() {
+ return getLastLogRecordAsyncInternal(false, false)
+ .map(RECORD_2_DLSN_FUNCTION);
+ }
+
+ /**
+ * Get the number of log records in the active portion of the log
+ * Any log segments that have already been truncated will not be included
+ *
+ * @return number of log records
+ * @throws IOException
+ */
+ @Override
+ public long getLogRecordCount() throws IOException {
+ checkClosedOrInError("getLogRecordCount");
+ return FutureUtils.result(getLogRecordCountAsync(DLSN.InitialDLSN));
+ }
+
+ /**
+ * Get the number of log records in the active portion of the log asynchronously.
+ * Any log segments that have already been truncated will not be included
+ *
+ * @return future number of log records
+ * @throws IOException
+ */
+ @Override
+ public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN) {
+ return processReaderOperation(new Function<BKLogReadHandler, Future<Long>>() {
+ @Override
+ public Future<Long> apply(BKLogReadHandler ledgerHandler) {
+ return ledgerHandler.asyncGetLogRecordCount(beginDLSN);
+ }
+ });
+ }
+
+ @Override
+ public void recover() throws IOException {
+ recoverInternal(conf.getUnpartitionedStreamName());
+ }
+
+ /**
+ * Recover a specified stream within the log container
+ * The writer implicitly recovers a topic when it resumes writing.
+ * This allows applications to recover a container explicitly so
+ * that application may read a fully recovered log before resuming
+ * the writes
+ *
+ * @throws IOException if the recovery fails
+ */
+ private void recoverInternal(String streamIdentifier) throws IOException {
+ checkClosedOrInError("recoverInternal");
+ BKLogWriteHandler ledgerHandler = createWriteHandler(true);
+ try {
+ FutureUtils.result(ledgerHandler.recoverIncompleteLogSegments());
+ } finally {
+ Utils.closeQuietly(ledgerHandler);
+ }
+ }
+
+ /**
+ * Delete all the partitions of the specified log
+ *
+ * @throws IOException if the deletion fails
+ */
+ @Override
+ public void delete() throws IOException {
+ FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+ .deleteLog(uri, getStreamName()));
+ }
+
+ /**
+ * The DistributedLogManager may archive/purge any logs for transactionId
+ * less than or equal to minImageTxId.
+ * This is to be used only when the client explicitly manages deletion. If
+ * the cleanup policy is based on sliding time window, then this method need
+ * not be called.
+ *
+ * @param minTxIdToKeep the earliest txid that must be retained
+ * @throws IOException if purging fails
+ */
+ @Override
+ public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException {
+ Preconditions.checkArgument(minTxIdToKeep > 0, "Invalid transaction id " + minTxIdToKeep);
+ checkClosedOrInError("purgeLogSegmentsOlderThan");
+ BKLogWriteHandler ledgerHandler = createWriteHandler(true);
+ try {
+ LOG.info("Purging logs for {} older than {}", ledgerHandler.getFullyQualifiedName(), minTxIdToKeep);
+ FutureUtils.result(ledgerHandler.purgeLogSegmentsOlderThanTxnId(minTxIdToKeep));
+ } finally {
+ Utils.closeQuietly(ledgerHandler);
+ }
+ }
+
+ static class PendingReaders implements AsyncCloseable {
+
+ final ExecutorService executorService;
+ final Set<AsyncCloseable> readers = new HashSet<AsyncCloseable>();
+
+ PendingReaders(ExecutorService executorService) {
+ this.executorService = executorService;
+ }
+
+ public synchronized void remove(AsyncCloseable reader) {
+ readers.remove(reader);
+ }
+
+ public synchronized void add(AsyncCloseable reader) {
+ readers.add(reader);
+ }
+
+ @Override
+ public Future<Void> asyncClose() {
+ return Utils.closeSequence(executorService, true, readers.toArray(new AsyncLogReader[readers.size()]))
+ .onSuccess(new AbstractFunction1<Void, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(Void value) {
+ readers.clear();
+ return BoxedUnit.UNIT;
+ }
+ });
+ }
+ };
+
+ /**
+ * Close the distributed log manager, freeing any resources it may hold.
+ */
+ @Override
+ public Future<Void> asyncClose() {
+ Promise<Void> closeFuture;
+ BKLogReadHandler readHandlerToClose;
+ synchronized (this) {
+ if (null != closePromise) {
+ return closePromise;
+ }
+ closeFuture = closePromise = new Promise<Void>();
+ readHandlerToClose = readHandlerForListener;
+ }
+
+ Future<Void> closeResult = Utils.closeSequence(null, true,
+ readHandlerToClose,
+ pendingReaders,
+ resourcesCloseable.or(AsyncCloseable.NULL));
+ closeResult.proxyTo(closeFuture);
+ return closeFuture;
+ }
+
+ @Override
+ public void close() throws IOException {
+ FutureUtils.result(asyncClose());
+ }
+
+ @Override
+ public String toString() {
+ return String.format("DLM:%s:%s", getUri(), getStreamName());
+ }
+
+ public void raiseAlert(String msg, Object... args) {
+ alertStatsLogger.raise(msg, args);
+ }
+
+ @Override
+ public SubscriptionsStore getSubscriptionsStore() {
+ return driver.getSubscriptionsStore(getStreamName());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
new file mode 100644
index 0000000..0a4608e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
@@ -0,0 +1,320 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Ticker;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.injector.AsyncFailureInjector;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.NamespaceDriver;
+import org.apache.distributedlog.util.ConfUtils;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.PermitLimiter;
+import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.distributedlog.util.Utils;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.distributedlog.namespace.NamespaceDriver.Role.WRITER;
+import static org.apache.distributedlog.util.DLUtils.validateName;
+
+/**
+ * BKDistributedLogNamespace is the default implementation of {@link DistributedLogNamespace}. It uses
+ * zookeeper for metadata storage and bookkeeper for data storage.
+ * <h3>Metrics</h3>
+ *
+ * <h4>ZooKeeper Client</h4>
+ * See {@link ZooKeeperClient} for detail sub-stats.
+ * <ul>
+ * <li> `scope`/dlzk_factory_writer_shared/* : stats about the zookeeper client shared by all DL writers.
+ * <li> `scope`/dlzk_factory_reader_shared/* : stats about the zookeeper client shared by all DL readers.
+ * <li> `scope`/bkzk_factory_writer_shared/* : stats about the zookeeper client used by bookkeeper client
+ * shared by all DL writers.
+ * <li> `scope`/bkzk_factory_reader_shared/* : stats about the zookeeper client used by bookkeeper client
+ * shared by all DL readers.
+ * </ul>
+ *
+ * <h4>BookKeeper Client</h4>
+ * BookKeeper client stats are exposed directly to current scope. See {@link BookKeeperClient} for detail stats.
+ *
+ * <h4>Utils</h4>
+ * <ul>
+ * <li> `scope`/factory/thread_pool/* : stats about the ordered scheduler used by this namespace.
+ * See {@link OrderedScheduler}.
+ * <li> `scope`/factory/readahead_thread_pool/* : stats about the readahead thread pool executor
+ * used by this namespace. See {@link MonitoredScheduledThreadPoolExecutor}.
+ * <li> `scope`/writeLimiter/* : stats about the global write limiter used by this namespace.
+ * See {@link PermitLimiter}.
+ * </ul>
+ *
+ * <h4>DistributedLogManager</h4>
+ *
+ * All the core stats about reader and writer are exposed under current scope via {@link BKDistributedLogManager}.
+ */
+public class BKDistributedLogNamespace implements DistributedLogNamespace {
+ static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogNamespace.class);
+
+ private final String clientId;
+ private final int regionId;
+ private final DistributedLogConfiguration conf;
+ private final URI namespace;
+ // namespace driver
+ private final NamespaceDriver driver;
+ // resources
+ private final OrderedScheduler scheduler;
+ private final PermitLimiter writeLimiter;
+ private final AsyncFailureInjector failureInjector;
+ // log segment metadata store
+ private final LogSegmentMetadataCache logSegmentMetadataCache;
+ // feature provider
+ private final FeatureProvider featureProvider;
+ // Stats Loggers
+ private final StatsLogger statsLogger;
+ private final StatsLogger perLogStatsLogger;
+
+ protected final AtomicBoolean closed = new AtomicBoolean(false);
+
+ public BKDistributedLogNamespace(
+ DistributedLogConfiguration conf,
+ URI uri,
+ NamespaceDriver driver,
+ OrderedScheduler scheduler,
+ FeatureProvider featureProvider,
+ PermitLimiter writeLimiter,
+ AsyncFailureInjector failureInjector,
+ StatsLogger statsLogger,
+ StatsLogger perLogStatsLogger,
+ String clientId,
+ 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;
+
+ // create a log segment metadata cache
+ this.logSegmentMetadataCache = new LogSegmentMetadataCache(conf, Ticker.systemTicker());
+ }
+
+ @Override
+ public NamespaceDriver getNamespaceDriver() {
+ return driver;
+ }
+
+ //
+ // Namespace Methods
+ //
+
+ @Override
+ public void createLog(String logName)
+ throws InvalidStreamNameException, IOException {
+ checkState();
+ validateName(logName);
+ URI uri = FutureUtils.result(driver.getLogMetadataStore().createLog(logName));
+ FutureUtils.result(driver.getLogStreamMetadataStore(WRITER).getLog(uri, logName, true, true));
+ }
+
+ @Override
+ public void deleteLog(String logName)
+ throws InvalidStreamNameException, LogNotFoundException, IOException {
+ checkState();
+ validateName(logName);
+ Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
+ if (!uri.isPresent()) {
+ throw new LogNotFoundException("Log " + logName + " isn't found.");
+ }
+ DistributedLogManager dlm = openLogInternal(
+ uri.get(),
+ logName,
+ Optional.<DistributedLogConfiguration>absent(),
+ Optional.<DynamicDistributedLogConfiguration>absent());
+ dlm.delete();
+ }
+
+ @Override
+ public DistributedLogManager openLog(String logName)
+ throws InvalidStreamNameException, IOException {
+ return openLog(logName,
+ Optional.<DistributedLogConfiguration>absent(),
+ Optional.<DynamicDistributedLogConfiguration>absent(),
+ Optional.<StatsLogger>absent());
+ }
+
+ @Override
+ public DistributedLogManager openLog(String logName,
+ Optional<DistributedLogConfiguration> logConf,
+ Optional<DynamicDistributedLogConfiguration> dynamicLogConf,
+ Optional<StatsLogger> perStreamStatsLogger)
+ throws InvalidStreamNameException, IOException {
+ checkState();
+ validateName(logName);
+ Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
+ if (!uri.isPresent()) {
+ throw new LogNotFoundException("Log " + logName + " isn't found.");
+ }
+ return openLogInternal(
+ uri.get(),
+ logName,
+ logConf,
+ dynamicLogConf);
+ }
+
+ @Override
+ public boolean logExists(String logName)
+ throws IOException, IllegalArgumentException {
+ checkState();
+ Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
+ if (uri.isPresent()) {
+ try {
+ FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+ .logExists(uri.get(), logName));
+ return true;
+ } catch (LogNotFoundException lnfe) {
+ return false;
+ }
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public Iterator<String> getLogs() throws IOException {
+ checkState();
+ return FutureUtils.result(driver.getLogMetadataStore().getLogs());
+ }
+
+ @Override
+ public void registerNamespaceListener(NamespaceListener listener) {
+ driver.getLogMetadataStore().registerNamespaceListener(listener);
+ }
+
+ @Override
+ public synchronized AccessControlManager createAccessControlManager() throws IOException {
+ checkState();
+ return driver.getAccessControlManager();
+ }
+
+ /**
+ * Open the log in location <i>uri</i>.
+ *
+ * @param uri
+ * location to store the log
+ * @param nameOfLogStream
+ * name of the log
+ * @param logConfiguration
+ * optional stream configuration
+ * @param dynamicLogConfiguration
+ * dynamic stream configuration overrides.
+ * @return distributedlog manager instance.
+ * @throws InvalidStreamNameException if the stream name is invalid
+ * @throws IOException
+ */
+ protected DistributedLogManager openLogInternal(
+ URI uri,
+ String nameOfLogStream,
+ Optional<DistributedLogConfiguration> logConfiguration,
+ Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration)
+ throws InvalidStreamNameException, IOException {
+ // Make sure the name is well formed
+ checkState();
+ validateName(nameOfLogStream);
+
+ DistributedLogConfiguration mergedConfiguration = new DistributedLogConfiguration();
+ mergedConfiguration.addConfiguration(conf);
+ mergedConfiguration.loadStreamConf(logConfiguration);
+ // If dynamic config was not provided, default to a static view of the global configuration.
+ DynamicDistributedLogConfiguration dynConf = null;
+ if (dynamicLogConfiguration.isPresent()) {
+ dynConf = dynamicLogConfiguration.get();
+ } else {
+ dynConf = ConfUtils.getConstDynConf(mergedConfiguration);
+ }
+
+ return new BKDistributedLogManager(
+ nameOfLogStream, /* Log Name */
+ mergedConfiguration, /* Configuration */
+ dynConf, /* Dynamic Configuration */
+ uri, /* Namespace URI */
+ driver, /* Namespace Driver */
+ logSegmentMetadataCache, /* Log Segment Metadata Cache */
+ scheduler, /* DL scheduler */
+ clientId, /* Client Id */
+ regionId, /* Region Id */
+ writeLimiter, /* Write Limiter */
+ featureProvider.scope("dl"), /* Feature Provider */
+ failureInjector, /* Failure Injector */
+ statsLogger, /* Stats Logger */
+ perLogStatsLogger, /* Per Log Stats Logger */
+ Optional.<AsyncCloseable>absent() /* shared resources, we don't need to close any resources in dlm */
+ );
+ }
+
+ /**
+ * Check the namespace state.
+ *
+ * @throws IOException
+ */
+ private void checkState() throws IOException {
+ if (closed.get()) {
+ LOG.error("BK namespace {} is already closed", namespace);
+ throw new AlreadyClosedException("BK namespace " + namespace + " is already closed");
+ }
+ }
+
+ /**
+ * Close the distributed log manager factory, freeing any resources it may hold.
+ */
+ @Override
+ public void close() {
+ if (!closed.compareAndSet(false, true)) {
+ return;
+ }
+ // shutdown the driver
+ Utils.close(driver);
+ // close the write limiter
+ this.writeLimiter.close();
+ // Shutdown the schedulers
+ SchedulerUtils.shutdownScheduler(scheduler, conf.getSchedulerShutdownTimeoutMs(),
+ TimeUnit.MILLISECONDS);
+ LOG.info("Executor Service Stopped.");
+ }
+}
[23/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
deleted file mode 100644
index 0c90a50..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
+++ /dev/null
@@ -1,374 +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.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;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.util.ZkUtils;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * ZooKeeper Based {@link com.twitter.distributedlog.acl.AccessControlManager}
- */
-public class ZKAccessControlManager implements AccessControlManager, Watcher {
-
- private static final Logger logger = LoggerFactory.getLogger(ZKAccessControlManager.class);
-
- private static final int ZK_RETRY_BACKOFF_MS = 500;
-
- protected final DistributedLogConfiguration conf;
- protected final ZooKeeperClient zkc;
- protected final String zkRootPath;
- protected final ScheduledExecutorService scheduledExecutorService;
-
- protected final ConcurrentMap<String, ZKAccessControl> streamEntries;
- protected ZKAccessControl defaultAccessControl;
- protected volatile boolean closed = false;
-
- public ZKAccessControlManager(DistributedLogConfiguration conf,
- ZooKeeperClient zkc,
- String zkRootPath,
- ScheduledExecutorService scheduledExecutorService) throws IOException {
- this.conf = conf;
- this.zkc = zkc;
- this.zkRootPath = zkRootPath;
- this.scheduledExecutorService = scheduledExecutorService;
- this.streamEntries = new ConcurrentHashMap<String, ZKAccessControl>();
- try {
- Await.result(fetchDefaultAccessControlEntry());
- } catch (Throwable t) {
- if (t instanceof InterruptedException) {
- throw new DLInterruptedException("Interrupted on getting default access control entry for " + zkRootPath, t);
- } else if (t instanceof KeeperException) {
- throw new IOException("Encountered zookeeper exception on getting default access control entry for " + zkRootPath, t);
- } else if (t instanceof IOException) {
- throw (IOException) t;
- } else {
- throw new IOException("Encountered unknown exception on getting access control entries for " + zkRootPath, t);
- }
- }
-
- try {
- Await.result(fetchAccessControlEntries());
- } catch (Throwable t) {
- if (t instanceof InterruptedException) {
- throw new DLInterruptedException("Interrupted on getting access control entries for " + zkRootPath, t);
- } else if (t instanceof KeeperException) {
- throw new IOException("Encountered zookeeper exception on getting access control entries for " + zkRootPath, t);
- } else if (t instanceof IOException) {
- throw (IOException) t;
- } else {
- throw new IOException("Encountered unknown exception on getting access control entries for " + zkRootPath, t);
- }
- }
- }
-
- protected AccessControlEntry getAccessControlEntry(String stream) {
- ZKAccessControl entry = streamEntries.get(stream);
- entry = null == entry ? defaultAccessControl : entry;
- return entry.getAccessControlEntry();
- }
-
- @Override
- public boolean allowWrite(String stream) {
- return !getAccessControlEntry(stream).isDenyWrite();
- }
-
- @Override
- public boolean allowTruncate(String stream) {
- return !getAccessControlEntry(stream).isDenyTruncate();
- }
-
- @Override
- public boolean allowDelete(String stream) {
- return !getAccessControlEntry(stream).isDenyDelete();
- }
-
- @Override
- public boolean allowAcquire(String stream) {
- return !getAccessControlEntry(stream).isDenyAcquire();
- }
-
- @Override
- public boolean allowRelease(String stream) {
- return !getAccessControlEntry(stream).isDenyRelease();
- }
-
- @Override
- public void close() {
- closed = true;
- }
-
- private Future<Void> fetchAccessControlEntries() {
- final Promise<Void> promise = new Promise<Void>();
- fetchAccessControlEntries(promise);
- return promise;
- }
-
- private void fetchAccessControlEntries(final Promise<Void> promise) {
- try {
- zkc.get().getChildren(zkRootPath, this, new AsyncCallback.Children2Callback() {
- @Override
- public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
- if (KeeperException.Code.OK.intValue() != rc) {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- return;
- }
- Set<String> streamsReceived = new HashSet<String>();
- streamsReceived.addAll(children);
- Set<String> streamsCached = streamEntries.keySet();
- Set<String> streamsRemoved = Sets.difference(streamsCached, streamsReceived).immutableCopy();
- for (String s : streamsRemoved) {
- ZKAccessControl accessControl = streamEntries.remove(s);
- if (null != accessControl) {
- logger.info("Removed Access Control Entry for stream {} : {}", s, accessControl.getAccessControlEntry());
- }
- }
- if (streamsReceived.isEmpty()) {
- promise.setValue(null);
- return;
- }
- final AtomicInteger numPendings = new AtomicInteger(streamsReceived.size());
- final AtomicInteger numFailures = new AtomicInteger(0);
- for (String s : streamsReceived) {
- final String streamName = s;
- ZKAccessControl.read(zkc, zkRootPath + "/" + streamName, null)
- .addEventListener(new FutureEventListener<ZKAccessControl>() {
-
- @Override
- public void onSuccess(ZKAccessControl accessControl) {
- streamEntries.put(streamName, accessControl);
- logger.info("Added overrided access control for stream {} : {}", streamName, accessControl.getAccessControlEntry());
- complete();
- }
-
- @Override
- public void onFailure(Throwable cause) {
- if (cause instanceof KeeperException.NoNodeException) {
- streamEntries.remove(streamName);
- } else if (cause instanceof ZKAccessControl.CorruptedAccessControlException) {
- logger.warn("Access control is corrupted for stream {} @ {}, skipped it ...",
- new Object[] { streamName, zkRootPath, cause });
- streamEntries.remove(streamName);
- } else {
- if (1 == numFailures.incrementAndGet()) {
- promise.setException(cause);
- }
- }
- complete();
- }
-
- private void complete() {
- if (0 == numPendings.decrementAndGet() && numFailures.get() == 0) {
- promise.setValue(null);
- }
- }
- });
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- } catch (InterruptedException e) {
- promise.setException(e);
- }
- }
-
- private Future<ZKAccessControl> fetchDefaultAccessControlEntry() {
- final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
- fetchDefaultAccessControlEntry(promise);
- return promise;
- }
-
- private void fetchDefaultAccessControlEntry(final Promise<ZKAccessControl> promise) {
- ZKAccessControl.read(zkc, zkRootPath, this)
- .addEventListener(new FutureEventListener<ZKAccessControl>() {
- @Override
- public void onSuccess(ZKAccessControl accessControl) {
- logger.info("Default Access Control will be changed from {} to {}",
- ZKAccessControlManager.this.defaultAccessControl,
- accessControl);
- ZKAccessControlManager.this.defaultAccessControl = accessControl;
- promise.setValue(accessControl);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- if (cause instanceof KeeperException.NoNodeException) {
- logger.info("Default Access Control is missing, creating one for {} ...", zkRootPath);
- createDefaultAccessControlEntryIfNeeded(promise);
- } else {
- promise.setException(cause);
- }
- }
- });
- }
-
- private void createDefaultAccessControlEntryIfNeeded(final Promise<ZKAccessControl> promise) {
- ZooKeeper zk;
- try {
- zk = zkc.get();
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- return;
- } catch (InterruptedException e) {
- promise.setException(e);
- return;
- }
- ZkUtils.asyncCreateFullPathOptimistic(zk, zkRootPath, new byte[0], zkc.getDefaultACL(),
- CreateMode.PERSISTENT, new AsyncCallback.StringCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, String name) {
- if (KeeperException.Code.OK.intValue() == rc) {
- logger.info("Created zk path {} for default ACL.", zkRootPath);
- fetchDefaultAccessControlEntry(promise);
- } else {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
- }, null);
- }
-
- private void refetchDefaultAccessControlEntry(final int delayMs) {
- if (closed) {
- return;
- }
- scheduledExecutorService.schedule(new Runnable() {
- @Override
- public void run() {
- fetchDefaultAccessControlEntry().addEventListener(new FutureEventListener<ZKAccessControl>() {
- @Override
- public void onSuccess(ZKAccessControl value) {
- // no-op
- }
- @Override
- public void onFailure(Throwable cause) {
- if (cause instanceof ZKAccessControl.CorruptedAccessControlException) {
- logger.warn("Default access control entry is corrupted, ignore this update : ", cause);
- return;
- }
-
- logger.warn("Encountered an error on refetching default access control entry, retrying in {} ms : ",
- ZK_RETRY_BACKOFF_MS, cause);
- refetchDefaultAccessControlEntry(ZK_RETRY_BACKOFF_MS);
- }
- });
- }
- }, delayMs, TimeUnit.MILLISECONDS);
- }
-
- private void refetchAccessControlEntries(final int delayMs) {
- if (closed) {
- return;
- }
- scheduledExecutorService.schedule(new Runnable() {
- @Override
- public void run() {
- fetchAccessControlEntries().addEventListener(new FutureEventListener<Void>() {
- @Override
- public void onSuccess(Void value) {
- // no-op
- }
- @Override
- public void onFailure(Throwable cause) {
- logger.warn("Encountered an error on refetching access control entries, retrying in {} ms : ",
- ZK_RETRY_BACKOFF_MS, cause);
- refetchAccessControlEntries(ZK_RETRY_BACKOFF_MS);
- }
- });
- }
- }, delayMs, TimeUnit.MILLISECONDS);
- }
-
- private void refetchAllAccessControlEntries(final int delayMs) {
- if (closed) {
- return;
- }
- scheduledExecutorService.schedule(new Runnable() {
- @Override
- public void run() {
- fetchDefaultAccessControlEntry().addEventListener(new FutureEventListener<ZKAccessControl>() {
- @Override
- public void onSuccess(ZKAccessControl value) {
- fetchAccessControlEntries().addEventListener(new FutureEventListener<Void>() {
- @Override
- public void onSuccess(Void value) {
- // no-op
- }
-
- @Override
- public void onFailure(Throwable cause) {
- logger.warn("Encountered an error on fetching all access control entries, retrying in {} ms : ",
- ZK_RETRY_BACKOFF_MS, cause);
- refetchAccessControlEntries(ZK_RETRY_BACKOFF_MS);
- }
- });
- }
-
- @Override
- public void onFailure(Throwable cause) {
- logger.warn("Encountered an error on refetching all access control entries, retrying in {} ms : ",
- ZK_RETRY_BACKOFF_MS, cause);
- refetchAllAccessControlEntries(ZK_RETRY_BACKOFF_MS);
- }
- });
- }
- }, delayMs, TimeUnit.MILLISECONDS);
- }
-
- @Override
- public void process(WatchedEvent event) {
- if (Event.EventType.None.equals(event.getType())) {
- if (event.getState() == Event.KeeperState.Expired) {
- refetchAllAccessControlEntries(0);
- }
- } else if (Event.EventType.NodeDataChanged.equals(event.getType())) {
- logger.info("Default ACL for {} is changed, refetching ...", zkRootPath);
- refetchDefaultAccessControlEntry(0);
- } else if (Event.EventType.NodeChildrenChanged.equals(event.getType())) {
- logger.info("List of ACLs for {} are changed, refetching ...", zkRootPath);
- refetchAccessControlEntries(0);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/federated/FederatedZKLogMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
deleted file mode 100644
index 0a8f28b..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
+++ /dev/null
@@ -1,760 +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.federated;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.google.common.base.Optional;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.exceptions.LogExistsException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.impl.ZKNamespaceWatcher;
-import com.twitter.distributedlog.metadata.LogMetadataStore;
-import com.twitter.distributedlog.namespace.NamespaceWatcher;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.Code;
-import org.apache.zookeeper.OpResult;
-import org.apache.zookeeper.Transaction;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * A Federated ZooKeeper Based Log Metadata Store.
- *
- * To Upgrade a simple ZKLogMetadataStore to FederatedZKLogMetadataStore, following steps should be taken in sequence:
- * a) deploy the new code with disabling createStreamsIfNotExists in all writer.
- * b) once all proxies disable the flag, update namespace binding to enable federated namespace.
- * c) restart writers to take federated namespace in place.
- *
- * NOTE: current federated namespace isn't optimized for deletion/creation. so don't use it in the workloads
- * that have lots of creations or deletions.
- */
-public class FederatedZKLogMetadataStore extends NamespaceWatcher implements LogMetadataStore, Watcher, Runnable,
- FutureEventListener<Set<URI>> {
-
- static final Logger logger = LoggerFactory.getLogger(FederatedZKLogMetadataStore.class);
-
- private final static String ZNODE_SUB_NAMESPACES = ".subnamespaces";
- private final static String SUB_NAMESPACE_PREFIX = "NS_";
-
- /**
- * Create the federated namespace.
- *
- * @param namespace
- * namespace to create
- * @param zkc
- * zookeeper client
- * @throws InterruptedException
- * @throws ZooKeeperClient.ZooKeeperConnectionException
- * @throws KeeperException
- */
- public static void createFederatedNamespace(URI namespace, ZooKeeperClient zkc)
- throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
- String zkSubNamespacesPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES;
- Utils.zkCreateFullPathOptimistic(zkc, zkSubNamespacesPath, new byte[0],
- zkc.getDefaultACL(), CreateMode.PERSISTENT);
- }
-
- /**
- * Represent a sub namespace inside the federated namespace.
- */
- class SubNamespace implements NamespaceListener {
- final URI uri;
- final ZKNamespaceWatcher watcher;
- Promise<Set<String>> logsFuture = new Promise<Set<String>>();
-
- SubNamespace(URI uri) {
- this.uri = uri;
- this.watcher = new ZKNamespaceWatcher(conf, uri, zkc, scheduler);
- this.watcher.registerListener(this);
- }
-
- void watch() {
- this.watcher.watchNamespaceChanges();
- }
-
- synchronized Future<Set<String>> getLogs() {
- return logsFuture;
- }
-
- @Override
- public void onStreamsChanged(Iterator<String> newLogsIter) {
- Set<String> newLogs = Sets.newHashSet(newLogsIter);
- Set<String> oldLogs = Sets.newHashSet();
-
- // update the sub namespace cache
- Promise<Set<String>> newLogsPromise;
- synchronized (this) {
- if (logsFuture.isDefined()) { // the promise is already satisfied
- try {
- oldLogs = FutureUtils.result(logsFuture);
- } catch (IOException e) {
- logger.error("Unexpected exception when getting logs from a satisified future of {} : ",
- uri, e);
- }
- logsFuture = new Promise<Set<String>>();
- }
-
- // update the reverse cache
- for (String logName : newLogs) {
- URI oldURI = log2Locations.putIfAbsent(logName, uri);
- if (null != oldURI && !Objects.equal(uri, oldURI)) {
- logger.error("Log {} is found duplicated in multiple locations : old location = {}," +
- " new location = {}", new Object[] { logName, oldURI, uri });
- duplicatedLogFound.set(true);
- }
- }
-
- // remove the gone streams
- Set<String> deletedLogs = Sets.difference(oldLogs, newLogs);
- for (String logName : deletedLogs) {
- log2Locations.remove(logName, uri);
- }
- newLogsPromise = logsFuture;
- }
- newLogsPromise.setValue(newLogs);
-
- // notify namespace changes
- notifyOnNamespaceChanges();
- }
- }
-
- final DistributedLogConfiguration conf;
- final URI namespace;
- final ZooKeeperClient zkc;
- final OrderedScheduler scheduler;
- final String zkSubnamespacesPath;
- final AtomicBoolean duplicatedLogFound = new AtomicBoolean(false);
- final AtomicReference<String> duplicatedLogName = new AtomicReference<String>(null);
- final AtomicReference<Integer> zkSubnamespacesVersion = new AtomicReference<Integer>(null);
-
- final int maxLogsPerSubnamespace;
- // sub namespaces
- final ConcurrentSkipListMap<URI, SubNamespace> subNamespaces;
- // map between log name and its location
- final ConcurrentMap<String, URI> log2Locations;
- // final
- final boolean forceCheckLogExistence;
-
- public FederatedZKLogMetadataStore(
- DistributedLogConfiguration conf,
- URI namespace,
- ZooKeeperClient zkc,
- OrderedScheduler scheduler) throws IOException {
- this.conf = conf;
- this.namespace = namespace;
- this.zkc = zkc;
- this.scheduler = scheduler;
- this.forceCheckLogExistence = conf.getFederatedCheckExistenceWhenCacheMiss();
- this.subNamespaces = new ConcurrentSkipListMap<URI, SubNamespace>();
- this.log2Locations = new ConcurrentHashMap<String, URI>();
- this.zkSubnamespacesPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES;
- this.maxLogsPerSubnamespace = conf.getFederatedMaxLogsPerSubnamespace();
-
- // fetch the sub namespace
- Set<URI> uris = FutureUtils.result(fetchSubNamespaces(this));
- for (URI uri : uris) {
- SubNamespace subNs = new SubNamespace(uri);
- if (null == subNamespaces.putIfAbsent(uri, subNs)) {
- subNs.watch();
- logger.info("Watched sub namespace {}", uri);
- }
- }
-
- logger.info("Federated ZK LogMetadataStore is initialized for {}", namespace);
- }
-
- private void scheduleTask(Runnable r, long ms) {
- if (duplicatedLogFound.get()) {
- logger.error("Scheduler is halted for federated namespace {} as duplicated log found",
- namespace);
- return;
- }
- try {
- scheduler.schedule(r, ms, TimeUnit.MILLISECONDS);
- } catch (RejectedExecutionException ree) {
- logger.error("Task {} scheduled in {} ms is rejected : ", new Object[]{r, ms, ree});
- }
- }
-
- private <T> Future<T> postStateCheck(Future<T> future) {
- final Promise<T> postCheckedPromise = new Promise<T>();
- future.addEventListener(new FutureEventListener<T>() {
- @Override
- public void onSuccess(T value) {
- if (duplicatedLogFound.get()) {
- postCheckedPromise.setException(new UnexpectedException("Duplicate log found under " + namespace));
- } else {
- postCheckedPromise.setValue(value);
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- postCheckedPromise.setException(cause);
- }
- });
- return postCheckedPromise;
- }
-
- //
- // SubNamespace Related Methods
- //
-
- @VisibleForTesting
- Set<URI> getSubnamespaces() {
- return subNamespaces.keySet();
- }
-
- @VisibleForTesting
- void removeLogFromCache(String logName) {
- log2Locations.remove(logName);
- }
-
- private URI getSubNamespaceURI(String ns) throws URISyntaxException {
- return new URI(
- namespace.getScheme(),
- namespace.getUserInfo(),
- namespace.getHost(),
- namespace.getPort(),
- namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES + "/" + ns,
- namespace.getQuery(),
- namespace.getFragment());
- }
-
- Future<Set<URI>> getCachedSubNamespaces() {
- Set<URI> nsSet = subNamespaces.keySet();
- return Future.value(nsSet);
- }
-
- Future<Set<URI>> fetchSubNamespaces(final Watcher watcher) {
- final Promise<Set<URI>> promise = new Promise<Set<URI>>();
- try {
- zkc.get().sync(this.zkSubnamespacesPath, new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx) {
- if (Code.OK.intValue() == rc) {
- fetchSubNamespaces(watcher, promise);
- } else {
- promise.setException(KeeperException.create(Code.get(rc)));
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- } catch (InterruptedException e) {
- promise.setException(e);
- }
- return promise;
- }
-
- private void fetchSubNamespaces(Watcher watcher,
- final Promise<Set<URI>> promise) {
- try {
- zkc.get().getChildren(this.zkSubnamespacesPath, watcher,
- new AsyncCallback.Children2Callback() {
- @Override
- public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
- if (Code.NONODE.intValue() == rc) {
- promise.setException(new UnexpectedException(
- "The subnamespaces don't exist for the federated namespace " + namespace));
- } else if (Code.OK.intValue() == rc) {
- Set<URI> subnamespaces = Sets.newHashSet();
- subnamespaces.add(namespace);
- try {
- for (String ns : children) {
- subnamespaces.add(getSubNamespaceURI(ns));
- }
- } catch (URISyntaxException use) {
- logger.error("Invalid sub namespace uri found : ", use);
- promise.setException(new UnexpectedException(
- "Invalid sub namespace uri found in " + namespace, use));
- return;
- }
- // update the sub namespaces set before update version
- setZkSubnamespacesVersion(stat.getVersion());
- promise.setValue(subnamespaces);
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- } catch (InterruptedException e) {
- promise.setException(e);
- }
- }
-
- @Override
- public void run() {
- fetchSubNamespaces(this).addEventListener(this);
- }
-
- @Override
- public void onSuccess(Set<URI> uris) {
- for (URI uri : uris) {
- if (subNamespaces.containsKey(uri)) {
- continue;
- }
- SubNamespace subNs = new SubNamespace(uri);
- if (null == subNamespaces.putIfAbsent(uri, subNs)) {
- subNs.watch();
- logger.info("Watched new sub namespace {}.", uri);
- notifyOnNamespaceChanges();
- }
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- // failed to fetch namespaces, retry later
- scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
- }
-
- @Override
- public void process(WatchedEvent watchedEvent) {
- if (Event.EventType.None == watchedEvent.getType() &&
- Event.KeeperState.Expired == watchedEvent.getState()) {
- scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
- return;
- }
- if (Event.EventType.NodeChildrenChanged == watchedEvent.getType()) {
- // fetch the namespace
- fetchSubNamespaces(this).addEventListener(this);
- }
- }
-
- //
- // Log Related Methods
- //
-
- private <A> Future<A> duplicatedLogException(String logName) {
- return Future.exception(new UnexpectedException("Duplicated log " + logName
- + " found in namespace " + namespace));
- }
-
- @Override
- public Future<URI> createLog(final String logName) {
- if (duplicatedLogFound.get()) {
- return duplicatedLogException(duplicatedLogName.get());
- }
- Promise<URI> createPromise = new Promise<URI>();
- doCreateLog(logName, createPromise);
- return postStateCheck(createPromise);
- }
-
- void doCreateLog(final String logName, final Promise<URI> createPromise) {
- getLogLocation(logName).addEventListener(new FutureEventListener<Optional<URI>>() {
- @Override
- public void onSuccess(Optional<URI> uriOptional) {
- if (uriOptional.isPresent()) {
- createPromise.setException(new LogExistsException("Log " + logName + " already exists in " + uriOptional.get()));
- } else {
- getCachedSubNamespacesAndCreateLog(logName, createPromise);
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- createPromise.setException(cause);
- }
- });
- }
-
- private void getCachedSubNamespacesAndCreateLog(final String logName,
- final Promise<URI> createPromise) {
- getCachedSubNamespaces().addEventListener(new FutureEventListener<Set<URI>>() {
- @Override
- public void onSuccess(Set<URI> uris) {
- findSubNamespaceToCreateLog(logName, uris, createPromise);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- createPromise.setException(cause);
- }
- });
- }
-
- private void fetchSubNamespacesAndCreateLog(final String logName,
- final Promise<URI> createPromise) {
- fetchSubNamespaces(null).addEventListener(new FutureEventListener<Set<URI>>() {
- @Override
- public void onSuccess(Set<URI> uris) {
- findSubNamespaceToCreateLog(logName, uris, createPromise);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- createPromise.setException(cause);
- }
- });
- }
-
- private void findSubNamespaceToCreateLog(final String logName,
- final Set<URI> uris,
- final Promise<URI> createPromise) {
- final List<URI> uriList = Lists.newArrayListWithExpectedSize(uris.size());
- List<Future<Set<String>>> futureList = Lists.newArrayListWithExpectedSize(uris.size());
- for (URI uri : uris) {
- SubNamespace subNs = subNamespaces.get(uri);
- if (null == subNs) {
- createPromise.setException(new UnexpectedException("No sub namespace " + uri + " found"));
- return;
- }
- futureList.add(subNs.getLogs());
- uriList.add(uri);
- }
- Future.collect(futureList).addEventListener(new FutureEventListener<List<Set<String>>>() {
- @Override
- public void onSuccess(List<Set<String>> resultList) {
- for (int i = resultList.size() - 1; i >= 0; i--) {
- Set<String> logs = resultList.get(i);
- if (logs.size() < maxLogsPerSubnamespace) {
- URI uri = uriList.get(i);
- createLogInNamespace(uri, logName, createPromise);
- return;
- }
- }
- // All sub namespaces are full
- createSubNamespace().addEventListener(new FutureEventListener<URI>() {
- @Override
- public void onSuccess(URI uri) {
- // the new namespace will be propagated to the namespace cache by the namespace listener
- // so we don't need to cache it here. we could go ahead to create the stream under this
- // namespace, as we are using sequential znode. we are mostly the first guy who create
- // the log under this namespace.
- createLogInNamespace(uri, logName, createPromise);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- createPromise.setException(cause);
- }
- });
- }
-
- @Override
- public void onFailure(Throwable cause) {
- createPromise.setException(cause);
- }
- });
- }
-
- private String getNamespaceFromZkPath(String zkPath) throws UnexpectedException {
- String parts[] = zkPath.split(SUB_NAMESPACE_PREFIX);
- if (parts.length <= 0) {
- throw new UnexpectedException("Invalid namespace @ " + zkPath);
- }
- return SUB_NAMESPACE_PREFIX + parts[parts.length - 1];
- }
-
- Future<URI> createSubNamespace() {
- final Promise<URI> promise = new Promise<URI>();
-
- final String nsPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES + "/" + SUB_NAMESPACE_PREFIX;
- try {
- zkc.get().create(nsPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT_SEQUENTIAL,
- new AsyncCallback.StringCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, String name) {
- if (Code.OK.intValue() == rc) {
- try {
- URI newUri = getSubNamespaceURI(getNamespaceFromZkPath(name));
- logger.info("Created sub namespace {}", newUri);
- promise.setValue(newUri);
- } catch (UnexpectedException ue) {
- promise.setException(ue);
- } catch (URISyntaxException e) {
- promise.setException(new UnexpectedException("Invalid namespace " + name + " is created."));
- }
- } else {
- promise.setException(KeeperException.create(Code.get(rc)));
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- promise.setException(e);
- } catch (InterruptedException e) {
- promise.setException(e);
- }
-
- return promise;
- }
-
- /**
- * Create a log under the namespace. To guarantee there is only one creation happens at time
- * in a federated namespace, we use CAS operation in zookeeper.
- *
- * @param uri
- * namespace
- * @param logName
- * name of the log
- * @param createPromise
- * the promise representing the creation result.
- */
- private void createLogInNamespace(final URI uri,
- final String logName,
- final Promise<URI> createPromise) {
- // TODO: rewrite this after we bump to zk 3.5, where we will have asynchronous version of multi
- scheduler.submit(new Runnable() {
- @Override
- public void run() {
- try {
- createLogInNamespaceSync(uri, logName);
- createPromise.setValue(uri);
- } catch (InterruptedException e) {
- createPromise.setException(e);
- } catch (IOException e) {
- createPromise.setException(e);
- } catch (KeeperException.BadVersionException bve) {
- fetchSubNamespacesAndCreateLog(logName, createPromise);
- } catch (KeeperException e) {
- createPromise.setException(e);
- }
- }
- });
- }
-
- void createLogInNamespaceSync(URI uri, String logName)
- throws InterruptedException, IOException, KeeperException {
- Transaction txn = zkc.get().transaction();
- // we don't have the zk version yet. set it to 0 instead of -1, to prevent non CAS operation.
- int zkVersion = null == zkSubnamespacesVersion.get() ? 0 : zkSubnamespacesVersion.get();
- txn.setData(zkSubnamespacesPath, uri.getPath().getBytes(UTF_8), zkVersion);
- String logPath = uri.getPath() + "/" + logName;
- txn.create(logPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT);
- try {
- txn.commit();
- // if the transaction succeed, the zk version is advanced
- setZkSubnamespacesVersion(zkVersion + 1);
- } catch (KeeperException ke) {
- List<OpResult> opResults = ke.getResults();
- OpResult createResult = opResults.get(1);
- if (createResult instanceof OpResult.ErrorResult) {
- OpResult.ErrorResult errorResult = (OpResult.ErrorResult) createResult;
- if (Code.NODEEXISTS.intValue() == errorResult.getErr()) {
- throw new LogExistsException("Log " + logName + " already exists");
- }
- }
- OpResult setResult = opResults.get(0);
- if (setResult instanceof OpResult.ErrorResult) {
- OpResult.ErrorResult errorResult = (OpResult.ErrorResult) setResult;
- if (Code.BADVERSION.intValue() == errorResult.getErr()) {
- throw KeeperException.create(Code.BADVERSION);
- }
- }
- throw new ZKException("ZK exception in creating log " + logName + " in " + uri, ke);
- }
- }
-
- void setZkSubnamespacesVersion(int zkVersion) {
- Integer oldVersion;
- boolean done = false;
- while (!done) {
- oldVersion = zkSubnamespacesVersion.get();
- if (null == oldVersion) {
- done = zkSubnamespacesVersion.compareAndSet(null, zkVersion);
- continue;
- }
- if (oldVersion < zkVersion) {
- done = zkSubnamespacesVersion.compareAndSet(oldVersion, zkVersion);
- continue;
- } else {
- done = true;
- }
- }
- }
-
- @Override
- public Future<Optional<URI>> getLogLocation(final String logName) {
- if (duplicatedLogFound.get()) {
- return duplicatedLogException(duplicatedLogName.get());
- }
- URI location = log2Locations.get(logName);
- if (null != location) {
- return postStateCheck(Future.value(Optional.of(location)));
- }
- if (!forceCheckLogExistence) {
- Optional<URI> result = Optional.absent();
- return Future.value(result);
- }
- return postStateCheck(fetchLogLocation(logName).onSuccess(
- new AbstractFunction1<Optional<URI>, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Optional<URI> uriOptional) {
- if (uriOptional.isPresent()) {
- log2Locations.putIfAbsent(logName, uriOptional.get());
- }
- return BoxedUnit.UNIT;
- }
- }));
- }
-
- private Future<Optional<URI>> fetchLogLocation(final String logName) {
- final Promise<Optional<URI>> fetchPromise = new Promise<Optional<URI>>();
-
- Set<URI> uris = subNamespaces.keySet();
- List<Future<Optional<URI>>> fetchFutures = Lists.newArrayListWithExpectedSize(uris.size());
- for (URI uri : uris) {
- fetchFutures.add(fetchLogLocation(uri, logName));
- }
- Future.collect(fetchFutures).addEventListener(new FutureEventListener<List<Optional<URI>>>() {
- @Override
- public void onSuccess(List<Optional<URI>> fetchResults) {
- Optional<URI> result = Optional.absent();
- for (Optional<URI> fetchResult : fetchResults) {
- if (result.isPresent()) {
- if (fetchResult.isPresent()) {
- logger.error("Log {} is found in multiple sub namespaces : {} & {}.",
- new Object[] { logName, result.get(), fetchResult.get() });
- duplicatedLogName.compareAndSet(null, logName);
- duplicatedLogFound.set(true);
- fetchPromise.setException(new UnexpectedException("Log " + logName
- + " is found in multiple sub namespaces : "
- + result.get() + " & " + fetchResult.get()));
- return;
- }
- } else {
- result = fetchResult;
- }
- }
- fetchPromise.setValue(result);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- fetchPromise.setException(cause);
- }
- });
- return fetchPromise;
- }
-
- private Future<Optional<URI>> fetchLogLocation(final URI uri, String logName) {
- final Promise<Optional<URI>> fetchPromise = new Promise<Optional<URI>>();
- final String logRootPath = uri.getPath() + "/" + logName;
- try {
- zkc.get().exists(logRootPath, false, new AsyncCallback.StatCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, Stat stat) {
- if (Code.OK.intValue() == rc) {
- fetchPromise.setValue(Optional.of(uri));
- } else if (Code.NONODE.intValue() == rc) {
- fetchPromise.setValue(Optional.<URI>absent());
- } else {
- fetchPromise.setException(KeeperException.create(Code.get(rc)));
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- fetchPromise.setException(e);
- } catch (InterruptedException e) {
- fetchPromise.setException(e);
- }
- return fetchPromise;
- }
-
- @Override
- public Future<Iterator<String>> getLogs() {
- if (duplicatedLogFound.get()) {
- return duplicatedLogException(duplicatedLogName.get());
- }
- return postStateCheck(retrieveLogs().map(
- new AbstractFunction1<List<Set<String>>, Iterator<String>>() {
- @Override
- public Iterator<String> apply(List<Set<String>> resultList) {
- return getIterator(resultList);
- }
- }));
- }
-
- private Future<List<Set<String>>> retrieveLogs() {
- Collection<SubNamespace> subNss = subNamespaces.values();
- List<Future<Set<String>>> logsList = Lists.newArrayListWithExpectedSize(subNss.size());
- for (SubNamespace subNs : subNss) {
- logsList.add(subNs.getLogs());
- }
- return Future.collect(logsList);
- }
-
- private Iterator<String> getIterator(List<Set<String>> resultList) {
- List<Iterator<String>> iterList = Lists.newArrayListWithExpectedSize(resultList.size());
- for (Set<String> result : resultList) {
- iterList.add(result.iterator());
- }
- return Iterators.concat(iterList.iterator());
- }
-
- @Override
- public void registerNamespaceListener(NamespaceListener listener) {
- registerListener(listener);
- }
-
- @Override
- protected void watchNamespaceChanges() {
- // as the federated namespace already started watching namespace changes,
- // we don't need to do any actions here
- }
-
- private void notifyOnNamespaceChanges() {
- retrieveLogs().onSuccess(new AbstractFunction1<List<Set<String>>, BoxedUnit>() {
- @Override
- public BoxedUnit apply(List<Set<String>> resultList) {
- for (NamespaceListener listener : listeners) {
- listener.onStreamsChanged(getIterator(resultList));
- }
- return BoxedUnit.UNIT;
- }
- });
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
----------------------------------------------------------------------
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
deleted file mode 100644
index d7ff4fb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
+++ /dev/null
@@ -1,85 +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.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();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
----------------------------------------------------------------------
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
deleted file mode 100644
index f85760d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
+++ /dev/null
@@ -1,837 +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.logsegment;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.EndOfLogSegmentException;
-import com.twitter.distributedlog.exceptions.ReadCancelledException;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryReader;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-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;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * BookKeeper ledger based log segment entry reader.
- */
-public class BKLogSegmentEntryReader implements Runnable, LogSegmentEntryReader, AsyncCallback.OpenCallback {
-
- private static final Logger logger = LoggerFactory.getLogger(BKLogSegmentEntryReader.class);
-
- private class CacheEntry implements Runnable, AsyncCallback.ReadCallback,
- AsyncCallback.ReadLastConfirmedAndEntryCallback {
-
- protected final long entryId;
- private boolean done;
- private LedgerEntry entry;
- private int rc;
-
- private CacheEntry(long entryId) {
- this.entryId = entryId;
- this.entry = null;
- this.rc = BKException.Code.UnexpectedConditionException;
- this.done = false;
- }
-
- long getEntryId() {
- return entryId;
- }
-
- synchronized boolean isDone() {
- return done;
- }
-
- void setValue(LedgerEntry entry) {
- synchronized (this) {
- if (done) {
- return;
- }
- this.rc = BKException.Code.OK;
- this.entry = entry;
- }
- setDone(true);
- }
-
- void setException(int rc) {
- synchronized (this) {
- if (done) {
- return;
- }
- this.rc = rc;
- }
- setDone(false);
- }
-
- void setDone(boolean success) {
- synchronized (this) {
- this.done = true;
- }
- onReadEntryDone(success);
- }
-
- synchronized boolean isSuccess() {
- return BKException.Code.OK == rc;
- }
-
- synchronized LedgerEntry getEntry() {
- return this.entry;
- }
-
- synchronized int getRc() {
- return rc;
- }
-
- @Override
- public void readComplete(int rc,
- LedgerHandle lh,
- Enumeration<LedgerEntry> entries,
- Object ctx) {
- if (failureInjector.shouldInjectCorruption(entryId, entryId)) {
- rc = BKException.Code.DigestMatchException;
- }
- processReadEntries(rc, lh, entries, ctx);
- }
-
- void processReadEntries(int rc,
- LedgerHandle lh,
- Enumeration<LedgerEntry> entries,
- Object ctx) {
- if (isDone()) {
- return;
- }
- if (!checkReturnCodeAndHandleFailure(rc, false)) {
- return;
- }
- LedgerEntry entry = null;
- while (entries.hasMoreElements()) {
- // more entries are returned
- if (null != entry) {
- setException(BKException.Code.UnexpectedConditionException);
- return;
- }
- entry = entries.nextElement();
- }
- if (null == entry || entry.getEntryId() != entryId) {
- setException(BKException.Code.UnexpectedConditionException);
- return;
- }
- setValue(entry);
- }
-
- @Override
- public void readLastConfirmedAndEntryComplete(int rc,
- long entryId,
- LedgerEntry entry,
- Object ctx) {
- if (failureInjector.shouldInjectCorruption(this.entryId, this.entryId)) {
- rc = BKException.Code.DigestMatchException;
- }
- processReadEntry(rc, entryId, entry, ctx);
- }
-
- void processReadEntry(int rc,
- long entryId,
- LedgerEntry entry,
- Object ctx) {
- if (isDone()) {
- return;
- }
- if (!checkReturnCodeAndHandleFailure(rc, true)) {
- return;
- }
- if (null != entry && this.entryId == entryId) {
- setValue(entry);
- return;
- }
- // the long poll is timeout or interrupted; we will retry it again.
- issueRead(this);
- }
-
- /**
- * Check return code and retry if needed.
- *
- * @param rc the return code
- * @param isLongPoll is it a long poll request
- * @return is the request successful or not
- */
- boolean checkReturnCodeAndHandleFailure(int rc, boolean isLongPoll) {
- if (BKException.Code.OK == rc) {
- numReadErrors.set(0);
- return true;
- }
- if (BKException.Code.BookieHandleNotAvailableException == rc ||
- (isLongPoll && BKException.Code.NoSuchLedgerExistsException == rc)) {
- int numErrors = Math.max(1, numReadErrors.incrementAndGet());
- int nextReadBackoffTime = Math.min(numErrors * readAheadWaitTime, maxReadBackoffTime);
- scheduler.schedule(
- getSegment().getLogSegmentId(),
- this,
- nextReadBackoffTime,
- TimeUnit.MILLISECONDS);
- } else {
- setException(rc);
- }
- return false;
- }
-
- @Override
- public void run() {
- issueRead(this);
- }
- }
-
- private class PendingReadRequest {
- private final int numEntries;
- private final List<Entry.Reader> entries;
- private final Promise<List<Entry.Reader>> promise;
-
- PendingReadRequest(int numEntries) {
- this.numEntries = numEntries;
- if (numEntries == 1) {
- this.entries = new ArrayList<Entry.Reader>(1);
- } else {
- this.entries = new ArrayList<Entry.Reader>();
- }
- this.promise = new Promise<List<Entry.Reader>>();
- }
-
- Promise<List<Entry.Reader>> getPromise() {
- return promise;
- }
-
- void setException(Throwable throwable) {
- FutureUtils.setException(promise, throwable);
- }
-
- void addEntry(Entry.Reader entry) {
- entries.add(entry);
- }
-
- void complete() {
- FutureUtils.setValue(promise, entries);
- onEntriesConsumed(entries.size());
- }
-
- boolean hasReadEntries() {
- return entries.size() > 0;
- }
-
- boolean hasReadEnoughEntries() {
- return entries.size() >= numEntries;
- }
- }
-
- private final BookKeeper bk;
- private final DistributedLogConfiguration conf;
- private final OrderedScheduler scheduler;
- private final long lssn;
- private final long startSequenceId;
- private final boolean envelopeEntries;
- private final boolean deserializeRecordSet;
- private final int numPrefetchEntries;
- private final int maxPrefetchEntries;
- // state
- private Promise<Void> closePromise = null;
- private LogSegmentMetadata metadata;
- private LedgerHandle lh;
- private final List<LedgerHandle> openLedgerHandles;
- private CacheEntry outstandingLongPoll;
- private long nextEntryId;
- 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;
- private final AtomicInteger numReadErrors = new AtomicInteger(0);
- private final boolean skipBrokenEntries;
- // readahead cache
- int cachedEntries = 0;
- int numOutstandingEntries = 0;
- final LinkedBlockingQueue<CacheEntry> readAheadEntries;
- // request queue
- final LinkedList<PendingReadRequest> readQueue;
-
- // failure injector
- private final AsyncFailureInjector failureInjector;
- // Stats
- private final Counter skippedBrokenEntriesCounter;
-
- BKLogSegmentEntryReader(LogSegmentMetadata metadata,
- LedgerHandle lh,
- long startEntryId,
- BookKeeper bk,
- OrderedScheduler scheduler,
- DistributedLogConfiguration conf,
- StatsLogger statsLogger,
- AsyncFailureInjector failureInjector) {
- this.metadata = metadata;
- this.lssn = metadata.getLogSegmentSequenceNumber();
- this.startSequenceId = metadata.getStartSequenceId();
- this.envelopeEntries = metadata.getEnvelopeEntries();
- this.deserializeRecordSet = conf.getDeserializeRecordSetOnReads();
- this.lh = lh;
- this.nextEntryId = Math.max(startEntryId, 0);
- this.bk = bk;
- this.conf = conf;
- this.numPrefetchEntries = conf.getNumPrefetchEntriesPerLogSegment();
- this.maxPrefetchEntries = conf.getMaxPrefetchEntriesPerLogSegment();
- this.scheduler = scheduler;
- this.openLedgerHandles = Lists.newArrayList();
- this.openLedgerHandles.add(lh);
- this.outstandingLongPoll = null;
- // create the readahead queue
- this.readAheadEntries = new LinkedBlockingQueue<CacheEntry>();
- // create the read request queue
- this.readQueue = new LinkedList<PendingReadRequest>();
- // read backoff settings
- this.readAheadWaitTime = conf.getReadAheadWaitTime();
- this.maxReadBackoffTime = 4 * conf.getReadAheadWaitTime();
- // other read settings
- this.skipBrokenEntries = conf.getReadAheadSkipBrokenEntries();
-
- // Failure Injection
- this.failureInjector = failureInjector;
- // Stats
- this.skippedBrokenEntriesCounter = statsLogger.getCounter("skipped_broken_entries");
- }
-
- @VisibleForTesting
- public synchronized CacheEntry getOutstandingLongPoll() {
- return outstandingLongPoll;
- }
-
- @VisibleForTesting
- LinkedBlockingQueue<CacheEntry> getReadAheadEntries() {
- return this.readAheadEntries;
- }
-
- synchronized LedgerHandle getLh() {
- return lh;
- }
-
- @Override
- public synchronized LogSegmentMetadata getSegment() {
- return metadata;
- }
-
- @VisibleForTesting
- synchronized long getNextEntryId() {
- return nextEntryId;
- }
-
- @Override
- public void start() {
- prefetchIfNecessary();
- }
-
- @Override
- public boolean hasCaughtUpOnInprogress() {
- 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
- //
-
- @Override
- public synchronized void onLogSegmentMetadataUpdated(LogSegmentMetadata segment) {
- if (metadata == segment ||
- LogSegmentMetadata.COMPARATOR.compare(metadata, segment) == 0 ||
- !(metadata.isInProgress() && !segment.isInProgress())) {
- return;
- }
- // segment is closed from inprogress, then re-open the log segment
- bk.asyncOpenLedger(
- segment.getLogSegmentId(),
- BookKeeper.DigestType.CRC32,
- conf.getBKDigestPW().getBytes(UTF_8),
- this,
- segment);
- }
-
- @Override
- public void openComplete(int rc, LedgerHandle lh, Object ctx) {
- LogSegmentMetadata segment = (LogSegmentMetadata) ctx;
- if (BKException.Code.OK != rc) {
- // fail current reader or retry opening the reader
- failOrRetryOpenLedger(rc, segment);
- return;
- }
- // switch to new ledger handle if the log segment is moved to completed.
- CacheEntry longPollRead = null;
- synchronized (this) {
- if (isClosed()) {
- lh.asyncClose(new AsyncCallback.CloseCallback() {
- @Override
- public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
- logger.debug("Close the open ledger {} since the log segment reader is already closed",
- lh.getId());
- }
- }, null);
- return;
- }
- this.metadata = segment;
- this.lh = lh;
- this.openLedgerHandles.add(lh);
- longPollRead = outstandingLongPoll;
- }
- if (null != longPollRead) {
- // reissue the long poll read when the log segment state is changed
- issueRead(longPollRead);
- }
- // notify readers
- notifyReaders();
- }
-
- private void failOrRetryOpenLedger(int rc, final LogSegmentMetadata segment) {
- if (isClosed()) {
- return;
- }
- if (isBeyondLastAddConfirmed()) {
- // if the reader is already caught up, let's fail the reader immediately
- // as we need to pull the latest metadata of this log segment.
- setException(new BKTransmitException("Failed to open ledger for reading log segment " + getSegment(), rc),
- true);
- return;
- }
- // the reader is still catching up, retry opening the log segment later
- scheduler.schedule(segment.getLogSegmentId(), new Runnable() {
- @Override
- public void run() {
- onLogSegmentMetadataUpdated(segment);
- }
- }, conf.getZKRetryBackoffStartMillis(), TimeUnit.MILLISECONDS);
- }
-
- //
- // Change the state of this reader
- //
-
- private boolean checkClosedOrInError() {
- if (null != lastException.get()) {
- cancelAllPendingReads(lastException.get());
- return true;
- }
- return false;
- }
-
- /**
- * Set the reader into error state with return code <i>rc</i>.
- *
- * @param throwable exception indicating the error
- * @param isBackground is the reader set exception by background reads or foreground reads
- */
- private void setException(Throwable throwable, boolean isBackground) {
- lastException.compareAndSet(null, throwable);
- if (isBackground) {
- notifyReaders();
- }
- }
-
- /**
- * Notify the readers with the state change.
- */
- private void notifyReaders() {
- processReadRequests();
- }
-
- private void cancelAllPendingReads(Throwable throwExc) {
- List<PendingReadRequest> requestsToCancel;
- synchronized (readQueue) {
- requestsToCancel = Lists.newArrayListWithExpectedSize(readQueue.size());
- requestsToCancel.addAll(readQueue);
- readQueue.clear();
- }
- for (PendingReadRequest request : requestsToCancel) {
- request.setException(throwExc);
- }
- }
-
- //
- // Background Read Operations
- //
-
- private void onReadEntryDone(boolean success) {
- // we successfully read an entry
- synchronized (this) {
- --numOutstandingEntries;
- }
- // notify reader that there is entry ready
- notifyReaders();
- // stop prefetch if we already encountered exceptions
- if (success) {
- prefetchIfNecessary();
- }
- }
-
- private void onEntriesConsumed(int numEntries) {
- synchronized (this) {
- cachedEntries -= numEntries;
- }
- prefetchIfNecessary();
- }
-
- private void prefetchIfNecessary() {
- List<CacheEntry> entriesToFetch;
- synchronized (this) {
- if (cachedEntries >= maxPrefetchEntries) {
- return;
- }
- // we don't have enough entries, do prefetch
- int numEntriesToFetch = numPrefetchEntries - numOutstandingEntries;
- if (numEntriesToFetch <= 0) {
- return;
- }
- entriesToFetch = new ArrayList<CacheEntry>(numEntriesToFetch);
- for (int i = 0; i < numEntriesToFetch; i++) {
- if (cachedEntries >= maxPrefetchEntries) {
- break;
- }
- if ((isLedgerClosed() && nextEntryId > getLastAddConfirmed()) ||
- (!isLedgerClosed() && nextEntryId > getLastAddConfirmed() + 1)) {
- break;
- }
- CacheEntry entry = new CacheEntry(nextEntryId);
- entriesToFetch.add(entry);
- readAheadEntries.add(entry);
- ++numOutstandingEntries;
- ++cachedEntries;
- ++nextEntryId;
- }
- }
- for (CacheEntry entry : entriesToFetch) {
- issueRead(entry);
- }
- }
-
-
- private void issueRead(CacheEntry cacheEntry) {
- if (isClosed()) {
- return;
- }
- if (isLedgerClosed()) {
- if (isNotBeyondLastAddConfirmed(cacheEntry.getEntryId())) {
- issueSimpleRead(cacheEntry);
- return;
- } else {
- // Reach the end of stream
- notifyReaders();
- }
- } else { // the ledger is still in progress
- if (isNotBeyondLastAddConfirmed(cacheEntry.getEntryId())) {
- issueSimpleRead(cacheEntry);
- } else {
- issueLongPollRead(cacheEntry);
- }
- }
- }
-
- private void issueSimpleRead(CacheEntry cacheEntry) {
- getLh().asyncReadEntries(cacheEntry.entryId, cacheEntry.entryId, cacheEntry, null);
- }
-
- private void issueLongPollRead(CacheEntry cacheEntry) {
- // register the read as outstanding reads
- synchronized (this) {
- this.outstandingLongPoll = cacheEntry;
- }
-
- if (!hasCaughtupOnInprogress) {
- hasCaughtupOnInprogress = true;
- notifyCaughtupOnInprogress();
- }
- getLh().asyncReadLastConfirmedAndEntry(
- cacheEntry.entryId,
- conf.getReadLACLongPollTimeout(),
- false,
- cacheEntry,
- null);
- }
-
- //
- // Foreground Read Operations
- //
-
- Entry.Reader processReadEntry(LedgerEntry entry) throws IOException {
- return Entry.newBuilder()
- .setLogSegmentInfo(lssn, startSequenceId)
- .setEntryId(entry.getEntryId())
- .setEnvelopeEntry(envelopeEntries)
- .deserializeRecordSet(deserializeRecordSet)
- .setInputStream(entry.getEntryInputStream())
- .buildReader();
- }
-
- @Override
- public Future<List<Entry.Reader>> readNext(int numEntries) {
- final PendingReadRequest readRequest = new PendingReadRequest(numEntries);
-
- if (checkClosedOrInError()) {
- readRequest.setException(lastException.get());
- } else {
- boolean wasQueueEmpty;
- synchronized (readQueue) {
- wasQueueEmpty = readQueue.isEmpty();
- readQueue.add(readRequest);
- }
- if (wasQueueEmpty) {
- processReadRequests();
- }
- }
- return readRequest.getPromise();
- }
-
- private void processReadRequests() {
- if (isClosed()) {
- // the reader is already closed.
- return;
- }
-
- long prevCount = scheduleCount.getAndIncrement();
- if (0 == prevCount) {
- scheduler.submit(getSegment().getLogSegmentId(), this);
- }
- }
-
- /**
- * The core function to propagate fetched entries to read requests
- */
- @Override
- public void run() {
- long scheduleCountLocal = scheduleCount.get();
- while (true) {
- PendingReadRequest nextRequest = null;
- synchronized (readQueue) {
- nextRequest = readQueue.peek();
- }
-
- // if read queue is empty, nothing to read, return
- if (null == nextRequest) {
- scheduleCount.set(0L);
- return;
- }
-
- // if the oldest pending promise is interrupted then we must
- // mark the reader in error and abort all pending reads since
- // we don't know the last consumed read
- if (null == lastException.get()) {
- if (nextRequest.getPromise().isInterrupted().isDefined()) {
- setException(new DLInterruptedException("Interrupted on reading log segment "
- + getSegment() + " : " + nextRequest.getPromise().isInterrupted().get()), false);
- }
- }
-
- // if the reader is in error state, stop read
- if (checkClosedOrInError()) {
- return;
- }
-
- // read entries from readahead cache to satisfy next read request
- readEntriesFromReadAheadCache(nextRequest);
-
- // check if we can satisfy the read request
- if (nextRequest.hasReadEntries()) {
- PendingReadRequest request;
- synchronized (readQueue) {
- request = readQueue.poll();
- }
- if (null != request && nextRequest == request) {
- request.complete();
- } else {
- DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at reading from "
- + getSegment());
- nextRequest.setException(ise);
- if (null != request) {
- request.setException(ise);
- }
- setException(ise, false);
- }
- } else {
- if (0 == scheduleCountLocal) {
- return;
- }
- scheduleCountLocal = scheduleCount.decrementAndGet();
- }
- }
- }
-
- private void readEntriesFromReadAheadCache(PendingReadRequest nextRequest) {
- while (!nextRequest.hasReadEnoughEntries()) {
- CacheEntry entry;
- boolean hitEndOfLogSegment;
- synchronized (this) {
- entry = readAheadEntries.peek();
- hitEndOfLogSegment = (null == entry) && isEndOfLogSegment();
- }
- // reach end of log segment
- if (hitEndOfLogSegment) {
- setException(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
- return;
- }
- if (null == entry) {
- return;
- }
- // entry is not complete yet.
- if (!entry.isDone()) {
- // we already reached end of the log segment
- if (isEndOfLogSegment(entry.getEntryId())) {
- setException(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
- }
- return;
- }
- if (entry.isSuccess()) {
- CacheEntry removedEntry = readAheadEntries.poll();
- if (entry != removedEntry) {
- DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at reading from "
- + getSegment());
- setException(ise, false);
- return;
- }
- try {
- nextRequest.addEntry(processReadEntry(entry.getEntry()));
- } catch (IOException e) {
- setException(e, false);
- return;
- }
- } else if (skipBrokenEntries && BKException.Code.DigestMatchException == entry.getRc()) {
- // skip this entry and move forward
- skippedBrokenEntriesCounter.inc();
- readAheadEntries.poll();
- continue;
- } else {
- setException(new BKTransmitException("Encountered issue on reading entry " + entry.getEntryId()
- + " @ log segment " + getSegment(), entry.getRc()), false);
- return;
- }
- }
- }
-
- //
- // State Management
- //
-
- private synchronized boolean isEndOfLogSegment() {
- return isEndOfLogSegment(nextEntryId);
- }
-
- private boolean isEndOfLogSegment(long entryId) {
- return isLedgerClosed() && entryId > getLastAddConfirmed();
- }
-
- @Override
- public synchronized boolean isBeyondLastAddConfirmed() {
- return isBeyondLastAddConfirmed(nextEntryId);
- }
-
- private boolean isBeyondLastAddConfirmed(long entryId) {
- return entryId > getLastAddConfirmed();
- }
-
- private boolean isNotBeyondLastAddConfirmed(long entryId) {
- return entryId <= getLastAddConfirmed();
- }
-
- private boolean isLedgerClosed() {
- return getLh().isClosed();
- }
-
- @Override
- public long getLastAddConfirmed() {
- return getLh().getLastAddConfirmed();
- }
-
- synchronized boolean isClosed() {
- return null != closePromise;
- }
-
- @Override
- public Future<Void> asyncClose() {
- final Promise<Void> closeFuture;
- ReadCancelledException exception;
- LedgerHandle[] lhsToClose;
- synchronized (this) {
- if (null != closePromise) {
- return closePromise;
- }
- closeFuture = closePromise = new Promise<Void>();
- lhsToClose = openLedgerHandles.toArray(new LedgerHandle[openLedgerHandles.size()]);
- // set the exception to cancel pending and subsequent reads
- exception = new ReadCancelledException(getSegment().getZNodeName(), "Reader was closed");
- setException(exception, false);
- }
-
- // cancel all pending reads
- cancelAllPendingReads(exception);
-
- // close all the open ledger
- BKUtils.closeLedgers(lhsToClose).proxyTo(closeFuture);
- return closeFuture;
- }
-}
[47/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogClientImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogClientImpl.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogClientImpl.java
deleted file mode 100644
index 1077cd0..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogClientImpl.java
+++ /dev/null
@@ -1,1200 +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.client;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordSetBuffer;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.client.ownership.OwnershipCache;
-import com.twitter.distributedlog.client.proxy.ClusterClient;
-import com.twitter.distributedlog.client.proxy.HostProvider;
-import com.twitter.distributedlog.client.proxy.ProxyClient;
-import com.twitter.distributedlog.client.proxy.ProxyClientManager;
-import com.twitter.distributedlog.client.proxy.ProxyListener;
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.client.routing.RoutingService.RoutingContext;
-import com.twitter.distributedlog.client.stats.ClientStats;
-import com.twitter.distributedlog.client.stats.OpStats;
-import com.twitter.distributedlog.exceptions.DLClientClosedException;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.ServiceUnavailableException;
-import com.twitter.distributedlog.exceptions.StreamUnavailableException;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.thrift.service.BulkWriteResponse;
-import com.twitter.distributedlog.thrift.service.HeartbeatOptions;
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import com.twitter.distributedlog.thrift.service.ServerStatus;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.distributedlog.thrift.service.WriteContext;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.util.ProtocolUtils;
-import com.twitter.finagle.CancelledRequestException;
-import com.twitter.finagle.ConnectionFailedException;
-import com.twitter.finagle.Failure;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.RequestTimeoutException;
-import com.twitter.finagle.ServiceException;
-import com.twitter.finagle.ServiceTimeoutException;
-import com.twitter.finagle.WriteException;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId;
-import com.twitter.util.Duration;
-import com.twitter.util.Function;
-import com.twitter.util.Function0;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import org.apache.thrift.TApplicationException;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timeout;
-import org.jboss.netty.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.collection.Seq;
-import scala.runtime.AbstractFunction1;
-
-
-/**
- * Implementation of distributedlog client.
- */
-public class DistributedLogClientImpl implements DistributedLogClient, MonitorServiceClient,
- RoutingService.RoutingListener, ProxyListener, HostProvider {
-
- private static final Logger logger = LoggerFactory.getLogger(DistributedLogClientImpl.class);
-
- private final String clientName;
- private final ClientId clientId;
- private final ClientConfig clientConfig;
- private final RoutingService routingService;
- private final ProxyClient.Builder clientBuilder;
- private final boolean streamFailfast;
- private final Pattern streamNameRegexPattern;
-
- // Timer
- private final HashedWheelTimer dlTimer;
-
- // region resolver
- private final RegionResolver regionResolver;
-
- // Ownership maintenance
- private final OwnershipCache ownershipCache;
- // Channel/Client management
- private final ProxyClientManager clientManager;
- // Cluster Client (for routing service)
- private final Optional<ClusterClient> clusterClient;
-
- // Close Status
- private boolean closed = false;
- private final ReentrantReadWriteLock closeLock =
- new ReentrantReadWriteLock();
-
- abstract class StreamOp implements TimerTask {
- final String stream;
-
- final AtomicInteger tries = new AtomicInteger(0);
- final RoutingContext routingContext = RoutingContext.of(regionResolver);
- final WriteContext ctx = new WriteContext();
- final Stopwatch stopwatch;
- final OpStats opStats;
- SocketAddress nextAddressToSend;
-
- StreamOp(final String stream, final OpStats opStats) {
- this.stream = stream;
- this.stopwatch = Stopwatch.createStarted();
- this.opStats = opStats;
- }
-
- boolean shouldTimeout() {
- long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
- return shouldTimeout(elapsedMs);
- }
-
- boolean shouldTimeout(long elapsedMs) {
- return clientConfig.getRequestTimeoutMs() > 0
- && elapsedMs >= clientConfig.getRequestTimeoutMs();
- }
-
- void send(SocketAddress address) {
- long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
- if (clientConfig.getMaxRedirects() > 0
- && tries.get() >= clientConfig.getMaxRedirects()) {
- fail(address, new RequestTimeoutException(Duration.fromMilliseconds(elapsedMs),
- "Exhausted max redirects in " + elapsedMs + " ms"));
- return;
- } else if (shouldTimeout(elapsedMs)) {
- fail(address, new RequestTimeoutException(Duration.fromMilliseconds(elapsedMs),
- "Exhausted max request timeout " + clientConfig.getRequestTimeoutMs()
- + " in " + elapsedMs + " ms"));
- return;
- }
- synchronized (this) {
- String addrStr = address.toString();
- if (ctx.isSetTriedHosts() && ctx.getTriedHosts().contains(addrStr)) {
- nextAddressToSend = address;
- dlTimer.newTimeout(this,
- Math.min(clientConfig.getRedirectBackoffMaxMs(),
- tries.get() * clientConfig.getRedirectBackoffStartMs()),
- TimeUnit.MILLISECONDS);
- } else {
- doSend(address);
- }
- }
- }
-
- abstract Future<ResponseHeader> sendRequest(ProxyClient sc);
-
- void doSend(SocketAddress address) {
- ctx.addToTriedHosts(address.toString());
- if (clientConfig.isChecksumEnabled()) {
- Long crc32 = computeChecksum();
- if (null != crc32) {
- ctx.setCrc32(crc32);
- }
- }
- tries.incrementAndGet();
- sendWriteRequest(address, this);
- }
-
- void beforeComplete(ProxyClient sc, ResponseHeader responseHeader) {
- ownershipCache.updateOwner(stream, sc.getAddress());
- }
-
- void complete(SocketAddress address) {
- stopwatch.stop();
- opStats.completeRequest(address,
- stopwatch.elapsed(TimeUnit.MICROSECONDS), tries.get());
- }
-
- void fail(SocketAddress address, Throwable t) {
- stopwatch.stop();
- opStats.failRequest(address,
- stopwatch.elapsed(TimeUnit.MICROSECONDS), tries.get());
- }
-
- Long computeChecksum() {
- return null;
- }
-
- @Override
- public synchronized void run(Timeout timeout) throws Exception {
- if (!timeout.isCancelled() && null != nextAddressToSend) {
- doSend(nextAddressToSend);
- } else {
- fail(null, new CancelledRequestException());
- }
- }
- }
-
- class BulkWriteOp extends StreamOp {
-
- final List<ByteBuffer> data;
- final ArrayList<Promise<DLSN>> results;
-
- BulkWriteOp(final String name, final List<ByteBuffer> data) {
- super(name, clientStats.getOpStats("bulk_write"));
- this.data = data;
-
- // This could take a while (relatively speaking) for very large inputs. We probably don't want
- // to go so large for other reasons though.
- this.results = new ArrayList<Promise<DLSN>>(data.size());
- for (int i = 0; i < data.size(); i++) {
- checkNotNull(data.get(i));
- this.results.add(new Promise<DLSN>());
- }
- }
-
- @Override
- Future<ResponseHeader> sendRequest(final ProxyClient sc) {
- return sc.getService().writeBulkWithContext(stream, data, ctx)
- .addEventListener(new FutureEventListener<BulkWriteResponse>() {
- @Override
- public void onSuccess(BulkWriteResponse response) {
- // For non-success case, the ResponseHeader handler (the caller) will handle it.
- // Note success in this case means no finagle errors have occurred
- // (such as finagle connection issues). In general code != SUCCESS means there's some error
- // reported by dlog service. The caller will handle such errors.
- if (response.getHeader().getCode() == StatusCode.SUCCESS) {
- beforeComplete(sc, response.getHeader());
- BulkWriteOp.this.complete(sc.getAddress(), response);
- if (response.getWriteResponses().size() == 0 && data.size() > 0) {
- logger.error("non-empty bulk write got back empty response without failure for stream {}",
- stream);
- }
- }
- }
- @Override
- public void onFailure(Throwable cause) {
- // Handled by the ResponseHeader listener (attached by the caller).
- }
- }).map(new AbstractFunction1<BulkWriteResponse, ResponseHeader>() {
- @Override
- public ResponseHeader apply(BulkWriteResponse response) {
- // We need to return the ResponseHeader to the caller's listener to process DLOG errors.
- return response.getHeader();
- }
- });
- }
-
- void complete(SocketAddress address, BulkWriteResponse bulkWriteResponse) {
- super.complete(address);
- Iterator<WriteResponse> writeResponseIterator = bulkWriteResponse.getWriteResponses().iterator();
- Iterator<Promise<DLSN>> resultIterator = results.iterator();
-
- // Fill in errors from thrift responses.
- while (resultIterator.hasNext() && writeResponseIterator.hasNext()) {
- Promise<DLSN> result = resultIterator.next();
- WriteResponse writeResponse = writeResponseIterator.next();
- if (StatusCode.SUCCESS == writeResponse.getHeader().getCode()) {
- result.setValue(DLSN.deserialize(writeResponse.getDlsn()));
- } else {
- result.setException(DLException.of(writeResponse.getHeader()));
- }
- }
-
- // Should never happen, but just in case so there's some record.
- if (bulkWriteResponse.getWriteResponses().size() != data.size()) {
- logger.error("wrong number of results, response = {} records = {}",
- bulkWriteResponse.getWriteResponses().size(), data.size());
- }
- }
-
- @Override
- void fail(SocketAddress address, Throwable t) {
-
- // StreamOp.fail is called to fail the overall request. In case of BulkWriteOp we take the request level
- // exception to apply to the first write. In fact for request level exceptions no request has ever been
- // attempted, but logically we associate the error with the first write.
- super.fail(address, t);
- Iterator<Promise<DLSN>> resultIterator = results.iterator();
-
- // Fail the first write with the batch level failure.
- if (resultIterator.hasNext()) {
- Promise<DLSN> result = resultIterator.next();
- result.setException(t);
- }
-
- // Fail the remaining writes as cancelled requests.
- while (resultIterator.hasNext()) {
- Promise<DLSN> result = resultIterator.next();
- result.setException(new CancelledRequestException());
- }
- }
-
- @SuppressWarnings("unchecked")
- List<Future<DLSN>> result() {
- return (List) results;
- }
- }
-
- abstract class AbstractWriteOp extends StreamOp {
-
- final Promise<WriteResponse> result = new Promise<WriteResponse>();
- Long crc32 = null;
-
- AbstractWriteOp(final String name, final OpStats opStats) {
- super(name, opStats);
- }
-
- void complete(SocketAddress address, WriteResponse response) {
- super.complete(address);
- result.setValue(response);
- }
-
- @Override
- void fail(SocketAddress address, Throwable t) {
- super.fail(address, t);
- result.setException(t);
- }
-
- @Override
- Long computeChecksum() {
- if (null == crc32) {
- crc32 = ProtocolUtils.streamOpCRC32(stream);
- }
- return crc32;
- }
-
- @Override
- Future<ResponseHeader> sendRequest(final ProxyClient sc) {
- return this.sendWriteRequest(sc).addEventListener(new FutureEventListener<WriteResponse>() {
- @Override
- public void onSuccess(WriteResponse response) {
- if (response.getHeader().getCode() == StatusCode.SUCCESS) {
- beforeComplete(sc, response.getHeader());
- AbstractWriteOp.this.complete(sc.getAddress(), response);
- }
- }
- @Override
- public void onFailure(Throwable cause) {
- // handled by the ResponseHeader listener
- }
- }).map(new AbstractFunction1<WriteResponse, ResponseHeader>() {
- @Override
- public ResponseHeader apply(WriteResponse response) {
- return response.getHeader();
- }
- });
- }
-
- abstract Future<WriteResponse> sendWriteRequest(ProxyClient sc);
- }
-
- class WriteOp extends AbstractWriteOp {
- final ByteBuffer data;
-
- WriteOp(final String name, final ByteBuffer data) {
- super(name, clientStats.getOpStats("write"));
- this.data = data;
- }
-
- @Override
- Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
- return sc.getService().writeWithContext(stream, data, ctx);
- }
-
- @Override
- Long computeChecksum() {
- if (null == crc32) {
- byte[] dataBytes = new byte[data.remaining()];
- data.duplicate().get(dataBytes);
- crc32 = ProtocolUtils.writeOpCRC32(stream, dataBytes);
- }
- return crc32;
- }
-
- Future<DLSN> result() {
- return result.map(new AbstractFunction1<WriteResponse, DLSN>() {
- @Override
- public DLSN apply(WriteResponse response) {
- return DLSN.deserialize(response.getDlsn());
- }
- });
- }
- }
-
- class TruncateOp extends AbstractWriteOp {
- final DLSN dlsn;
-
- TruncateOp(String name, DLSN dlsn) {
- super(name, clientStats.getOpStats("truncate"));
- this.dlsn = dlsn;
- }
-
- @Override
- Long computeChecksum() {
- if (null == crc32) {
- crc32 = ProtocolUtils.truncateOpCRC32(stream, dlsn);
- }
- return crc32;
- }
-
- @Override
- Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
- return sc.getService().truncate(stream, dlsn.serialize(), ctx);
- }
-
- Future<Boolean> result() {
- return result.map(new AbstractFunction1<WriteResponse, Boolean>() {
- @Override
- public Boolean apply(WriteResponse response) {
- return true;
- }
- });
- }
- }
-
- class WriteRecordSetOp extends WriteOp {
-
- WriteRecordSetOp(String name, LogRecordSetBuffer recordSet) {
- super(name, recordSet.getBuffer());
- ctx.setIsRecordSet(true);
- }
-
- }
-
-
- class ReleaseOp extends AbstractWriteOp {
-
- ReleaseOp(String name) {
- super(name, clientStats.getOpStats("release"));
- }
-
- @Override
- Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
- return sc.getService().release(stream, ctx);
- }
-
- @Override
- void beforeComplete(ProxyClient sc, ResponseHeader header) {
- ownershipCache.removeOwnerFromStream(stream, sc.getAddress(), "Stream Deleted");
- }
-
- Future<Void> result() {
- return result.map(new AbstractFunction1<WriteResponse, Void>() {
- @Override
- public Void apply(WriteResponse response) {
- return null;
- }
- });
- }
- }
-
- class DeleteOp extends AbstractWriteOp {
-
- DeleteOp(String name) {
- super(name, clientStats.getOpStats("delete"));
- }
-
- @Override
- Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
- return sc.getService().delete(stream, ctx);
- }
-
- @Override
- void beforeComplete(ProxyClient sc, ResponseHeader header) {
- ownershipCache.removeOwnerFromStream(stream, sc.getAddress(), "Stream Deleted");
- }
-
- Future<Void> result() {
- return result.map(new AbstractFunction1<WriteResponse, Void>() {
- @Override
- public Void apply(WriteResponse v1) {
- return null;
- }
- });
- }
- }
-
- class CreateOp extends AbstractWriteOp {
-
- CreateOp(String name) {
- super(name, clientStats.getOpStats("create"));
- }
-
- @Override
- Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
- return sc.getService().create(stream, ctx);
- }
-
- @Override
- void beforeComplete(ProxyClient sc, ResponseHeader header) {
- ownershipCache.updateOwner(stream, sc.getAddress());
- }
-
- Future<Void> result() {
- return result.map(new AbstractFunction1<WriteResponse, Void>() {
- @Override
- public Void apply(WriteResponse v1) {
- return null;
- }
- }).voided();
- }
- }
-
- class HeartbeatOp extends AbstractWriteOp {
- HeartbeatOptions options;
-
- HeartbeatOp(String name, boolean sendReaderHeartBeat) {
- super(name, clientStats.getOpStats("heartbeat"));
- options = new HeartbeatOptions();
- options.setSendHeartBeatToReader(sendReaderHeartBeat);
- }
-
- @Override
- Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
- return sc.getService().heartbeatWithOptions(stream, ctx, options);
- }
-
- Future<Void> result() {
- return result.map(new AbstractFunction1<WriteResponse, Void>() {
- @Override
- public Void apply(WriteResponse response) {
- return null;
- }
- });
- }
- }
-
- // Stats
- private final ClientStats clientStats;
-
- public DistributedLogClientImpl(String name,
- ClientId clientId,
- RoutingService routingService,
- ClientBuilder clientBuilder,
- ClientConfig clientConfig,
- Optional<ClusterClient> clusterClient,
- StatsReceiver statsReceiver,
- StatsReceiver streamStatsReceiver,
- RegionResolver regionResolver,
- boolean enableRegionStats) {
- this.clientName = name;
- this.clientId = clientId;
- this.routingService = routingService;
- this.clientConfig = clientConfig;
- this.streamFailfast = clientConfig.getStreamFailfast();
- this.streamNameRegexPattern = Pattern.compile(clientConfig.getStreamNameRegex());
- this.regionResolver = regionResolver;
- // Build the timer
- this.dlTimer = new HashedWheelTimer(
- new ThreadFactoryBuilder().setNameFormat("DLClient-" + name + "-timer-%d").build(),
- this.clientConfig.getRedirectBackoffStartMs(),
- TimeUnit.MILLISECONDS);
- // register routing listener
- this.routingService.registerListener(this);
- // build the ownership cache
- this.ownershipCache = new OwnershipCache(this.clientConfig, this.dlTimer, statsReceiver, streamStatsReceiver);
- // Client Stats
- this.clientStats = new ClientStats(statsReceiver, enableRegionStats, regionResolver);
- // Client Manager
- this.clientBuilder = ProxyClient.newBuilder(clientName, clientId, clientBuilder, clientConfig, clientStats);
- this.clientManager = new ProxyClientManager(
- this.clientConfig, // client config
- this.clientBuilder, // client builder
- this.dlTimer, // timer
- this, // host provider
- clientStats); // client stats
- this.clusterClient = clusterClient;
- this.clientManager.registerProxyListener(this);
-
- // Cache Stats
- StatsReceiver cacheStatReceiver = statsReceiver.scope("cache");
- Seq<String> numCachedStreamsGaugeName =
- scala.collection.JavaConversions.asScalaBuffer(Arrays.asList("num_streams")).toList();
- cacheStatReceiver.provideGauge(numCachedStreamsGaugeName, new Function0<Object>() {
- @Override
- public Object apply() {
- return (float) ownershipCache.getNumCachedStreams();
- }
- });
- Seq<String> numCachedHostsGaugeName =
- scala.collection.JavaConversions.asScalaBuffer(Arrays.asList("num_hosts")).toList();
- cacheStatReceiver.provideGauge(numCachedHostsGaugeName, new Function0<Object>() {
- @Override
- public Object apply() {
- return (float) clientManager.getNumProxies();
- }
- });
-
- logger.info("Build distributedlog client : name = {}, client_id = {}, routing_service = {},"
- + " stats_receiver = {}, thriftmux = {}",
- new Object[] {
- name,
- clientId,
- routingService.getClass(),
- statsReceiver.getClass(),
- clientConfig.getThriftMux()
- });
- }
-
- @Override
- public Set<SocketAddress> getHosts() {
- Set<SocketAddress> hosts = Sets.newHashSet();
- // if using server side routing, we only handshake with the hosts in ownership cache.
- if (!clusterClient.isPresent()) {
- hosts.addAll(this.routingService.getHosts());
- }
- hosts.addAll(this.ownershipCache.getStreamOwnershipDistribution().keySet());
- return hosts;
- }
-
- @Override
- public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
- if (null != serverInfo
- && serverInfo.isSetServerStatus()
- && ServerStatus.DOWN == serverInfo.getServerStatus()) {
- logger.info("{} is detected as DOWN during handshaking", address);
- // server is shutting down
- handleServiceUnavailable(address, client, Optional.<StreamOp>absent());
- return;
- }
-
- if (null != serverInfo && serverInfo.isSetOwnerships()) {
- Map<String, String> ownerships = serverInfo.getOwnerships();
- logger.debug("Handshaked with {} : {} ownerships returned.", address, ownerships.size());
- for (Map.Entry<String, String> entry : ownerships.entrySet()) {
- Matcher matcher = streamNameRegexPattern.matcher(entry.getKey());
- if (!matcher.matches()) {
- continue;
- }
- updateOwnership(entry.getKey(), entry.getValue());
- }
- } else {
- logger.debug("Handshaked with {} : no ownerships returned", address);
- }
- }
-
- @Override
- public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
- cause = showRootCause(Optional.<StreamOp>absent(), cause);
- handleRequestException(address, client, Optional.<StreamOp>absent(), cause);
- }
-
- @VisibleForTesting
- public void handshake() {
- clientManager.handshake();
- logger.info("Handshaked with {} hosts, cached {} streams",
- clientManager.getNumProxies(), ownershipCache.getNumCachedStreams());
- }
-
- @Override
- public void onServerLeft(SocketAddress address) {
- onServerLeft(address, null);
- }
-
- private void onServerLeft(SocketAddress address, ProxyClient sc) {
- ownershipCache.removeAllStreamsFromOwner(address);
- if (null == sc) {
- clientManager.removeClient(address);
- } else {
- clientManager.removeClient(address, sc);
- }
- }
-
- @Override
- public void onServerJoin(SocketAddress address) {
- // we only pre-create connection for client-side routing
- // if it is server side routing, we only know the exact proxy address
- // when #getOwner.
- if (!clusterClient.isPresent()) {
- clientManager.createClient(address);
- }
- }
-
- public void close() {
- closeLock.writeLock().lock();
- try {
- if (closed) {
- return;
- }
- closed = true;
- } finally {
- closeLock.writeLock().unlock();
- }
- clientManager.close();
- routingService.unregisterListener(this);
- routingService.stopService();
- dlTimer.stop();
- }
-
- @Override
- public Future<Void> check(String stream) {
- final HeartbeatOp op = new HeartbeatOp(stream, false);
- sendRequest(op);
- return op.result();
- }
-
- @Override
- public Future<Void> heartbeat(String stream) {
- final HeartbeatOp op = new HeartbeatOp(stream, true);
- sendRequest(op);
- return op.result();
- }
-
- @Override
- public Map<SocketAddress, Set<String>> getStreamOwnershipDistribution() {
- return ownershipCache.getStreamOwnershipDistribution();
- }
-
- @Override
- public Future<Void> setAcceptNewStream(boolean enabled) {
- Map<SocketAddress, ProxyClient> snapshot = clientManager.getAllClients();
- List<Future<Void>> futures = new ArrayList<Future<Void>>(snapshot.size());
- for (Map.Entry<SocketAddress, ProxyClient> entry : snapshot.entrySet()) {
- futures.add(entry.getValue().getService().setAcceptNewStream(enabled));
- }
- return Future.collect(futures).map(new Function<List<Void>, Void>() {
- @Override
- public Void apply(List<Void> list) {
- return null;
- }
- });
- }
-
- @Override
- public Future<DLSN> write(String stream, ByteBuffer data) {
- final WriteOp op = new WriteOp(stream, data);
- sendRequest(op);
- return op.result();
- }
-
- @Override
- public Future<DLSN> writeRecordSet(String stream, final LogRecordSetBuffer recordSet) {
- final WriteRecordSetOp op = new WriteRecordSetOp(stream, recordSet);
- sendRequest(op);
- return op.result();
- }
-
- @Override
- public List<Future<DLSN>> writeBulk(String stream, List<ByteBuffer> data) {
- if (data.size() > 0) {
- final BulkWriteOp op = new BulkWriteOp(stream, data);
- sendRequest(op);
- return op.result();
- } else {
- return Collections.emptyList();
- }
- }
-
- @Override
- public Future<Boolean> truncate(String stream, DLSN dlsn) {
- final TruncateOp op = new TruncateOp(stream, dlsn);
- sendRequest(op);
- return op.result();
- }
-
- @Override
- public Future<Void> delete(String stream) {
- final DeleteOp op = new DeleteOp(stream);
- sendRequest(op);
- return op.result();
- }
-
- @Override
- public Future<Void> release(String stream) {
- final ReleaseOp op = new ReleaseOp(stream);
- sendRequest(op);
- return op.result();
- }
-
- @Override
- public Future<Void> create(String stream) {
- final CreateOp op = new CreateOp(stream);
- sendRequest(op);
- return op.result();
- }
-
- private void sendRequest(final StreamOp op) {
- closeLock.readLock().lock();
- try {
- if (closed) {
- op.fail(null, new DLClientClosedException("Client " + clientName + " is closed."));
- } else {
- doSend(op, null);
- }
- } finally {
- closeLock.readLock().unlock();
- }
- }
-
- /**
- * Send the stream operation by routing service, excluding previous address if it is not null.
- *
- * @param op
- * stream operation.
- * @param previousAddr
- * previous tried address.
- */
- private void doSend(final StreamOp op, final SocketAddress previousAddr) {
- if (null != previousAddr) {
- op.routingContext.addTriedHost(previousAddr, StatusCode.WRITE_EXCEPTION);
- }
- // Get host first
- final SocketAddress address = ownershipCache.getOwner(op.stream);
- if (null == address || op.routingContext.isTriedHost(address)) {
- getOwner(op).addEventListener(new FutureEventListener<SocketAddress>() {
- @Override
- public void onFailure(Throwable cause) {
- op.fail(null, cause);
- }
-
- @Override
- public void onSuccess(SocketAddress ownerAddr) {
- op.send(ownerAddr);
- }
- });
- } else {
- op.send(address);
- }
- }
-
- private void retryGetOwnerFromResourcePlacementServer(final StreamOp op,
- final Promise<SocketAddress> getOwnerPromise,
- final Throwable cause) {
- if (op.shouldTimeout()) {
- op.fail(null, cause);
- return;
- }
- getOwnerFromResourcePlacementServer(op, getOwnerPromise);
- }
-
- private void getOwnerFromResourcePlacementServer(final StreamOp op,
- final Promise<SocketAddress> getOwnerPromise) {
- clusterClient.get().getService().getOwner(op.stream, op.ctx)
- .addEventListener(new FutureEventListener<WriteResponse>() {
- @Override
- public void onFailure(Throwable cause) {
- getOwnerPromise.updateIfEmpty(new Throw<SocketAddress>(cause));
- }
-
- @Override
- public void onSuccess(WriteResponse value) {
- if (StatusCode.FOUND == value.getHeader().getCode()
- && null != value.getHeader().getLocation()) {
- try {
- InetSocketAddress addr = DLSocketAddress.deserialize(
- value.getHeader().getLocation()
- ).getSocketAddress();
- getOwnerPromise.updateIfEmpty(new Return<SocketAddress>(addr));
- } catch (IOException e) {
- // retry from the routing server again
- logger.error("ERROR in getOwner", e);
- retryGetOwnerFromResourcePlacementServer(op, getOwnerPromise, e);
- return;
- }
- } else {
- // retry from the routing server again
- retryGetOwnerFromResourcePlacementServer(op, getOwnerPromise,
- new StreamUnavailableException("Stream " + op.stream + "'s owner is unknown"));
- }
- }
- });
- }
-
- private Future<SocketAddress> getOwner(final StreamOp op) {
- if (clusterClient.isPresent()) {
- final Promise<SocketAddress> getOwnerPromise = new Promise<SocketAddress>();
- getOwnerFromResourcePlacementServer(op, getOwnerPromise);
- return getOwnerPromise;
- }
- // pickup host by hashing
- try {
- return Future.value(routingService.getHost(op.stream, op.routingContext));
- } catch (NoBrokersAvailableException nbae) {
- return Future.exception(nbae);
- }
- }
-
- private void sendWriteRequest(final SocketAddress addr, final StreamOp op) {
- // Get corresponding finagle client
- final ProxyClient sc = clientManager.getClient(addr);
- final long startTimeNanos = System.nanoTime();
- // write the request to that host.
- op.sendRequest(sc).addEventListener(new FutureEventListener<ResponseHeader>() {
- @Override
- public void onSuccess(ResponseHeader header) {
- if (logger.isDebugEnabled()) {
- logger.debug("Received response; header: {}", header);
- }
- clientStats.completeProxyRequest(addr, header.getCode(), startTimeNanos);
- // update routing context
- op.routingContext.addTriedHost(addr, header.getCode());
- switch (header.getCode()) {
- case SUCCESS:
- // success handling is done per stream op
- break;
- case FOUND:
- handleRedirectResponse(header, op, addr);
- break;
- // for overcapacity, dont report failure since this normally happens quite a bit
- case OVER_CAPACITY:
- logger.debug("Failed to write request to {} : {}", op.stream, header);
- op.fail(addr, DLException.of(header));
- break;
- // for responses that indicate the requests definitely failed,
- // we should fail them immediately (e.g. TOO_LARGE_RECORD, METADATA_EXCEPTION)
- case NOT_IMPLEMENTED:
- case METADATA_EXCEPTION:
- case LOG_EMPTY:
- case LOG_NOT_FOUND:
- case TRUNCATED_TRANSACTION:
- case END_OF_STREAM:
- case TRANSACTION_OUT_OF_ORDER:
- case INVALID_STREAM_NAME:
- case REQUEST_DENIED:
- case TOO_LARGE_RECORD:
- case CHECKSUM_FAILED:
- // status code NOT_READY is returned if failfast is enabled in the server. don't redirect
- // since the proxy may still own the stream.
- case STREAM_NOT_READY:
- op.fail(addr, DLException.of(header));
- break;
- case SERVICE_UNAVAILABLE:
- handleServiceUnavailable(addr, sc, Optional.of(op));
- break;
- case REGION_UNAVAILABLE:
- // region is unavailable, redirect the request to hosts in other region
- redirect(op, null);
- break;
- // Proxy was overloaded and refused to try to acquire the stream. Don't remove ownership, since
- // we didn't have it in the first place.
- case TOO_MANY_STREAMS:
- handleRedirectableError(addr, op, header);
- break;
- case STREAM_UNAVAILABLE:
- case ZOOKEEPER_ERROR:
- case LOCKING_EXCEPTION:
- case UNEXPECTED:
- case INTERRUPTED:
- case BK_TRANSMIT_ERROR:
- case FLUSH_TIMEOUT:
- default:
- // when we are receiving these exceptions from proxy, it means proxy or the stream is closed
- // redirect the request.
- ownershipCache.removeOwnerFromStream(op.stream, addr, header.getCode().name());
- handleRedirectableError(addr, op, header);
- break;
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- Optional<StreamOp> opOptional = Optional.of(op);
- cause = showRootCause(opOptional, cause);
- clientStats.failProxyRequest(addr, cause, startTimeNanos);
- handleRequestException(addr, sc, opOptional, cause);
- }
- });
- }
-
- // Response Handlers
-
- Throwable showRootCause(Optional<StreamOp> op, Throwable cause) {
- if (cause instanceof Failure) {
- Failure failure = (Failure) cause;
- if (failure.isFlagged(Failure.Wrapped())) {
- try {
- // if it is a wrapped failure, unwrap it first
- cause = failure.show();
- } catch (IllegalArgumentException iae) {
- if (op.isPresent()) {
- logger.warn("Failed to unwrap finagle failure of stream {} : ", op.get().stream, iae);
- } else {
- logger.warn("Failed to unwrap finagle failure : ", iae);
- }
- }
- }
- }
- return cause;
- }
-
- private void handleRedirectableError(SocketAddress addr,
- StreamOp op,
- ResponseHeader header) {
- if (streamFailfast) {
- op.fail(addr, DLException.of(header));
- } else {
- redirect(op, null);
- }
- }
-
- void handleServiceUnavailable(SocketAddress addr,
- ProxyClient sc,
- Optional<StreamOp> op) {
- // service is unavailable, remove it out of routing service
- routingService.removeHost(addr, new ServiceUnavailableException(addr + " is unavailable now."));
- onServerLeft(addr);
- if (op.isPresent()) {
- ownershipCache.removeOwnerFromStream(op.get().stream, addr, addr + " is unavailable now.");
- // redirect the request to other host.
- redirect(op.get(), null);
- }
- }
-
- void handleRequestException(SocketAddress addr,
- ProxyClient sc,
- Optional<StreamOp> op,
- Throwable cause) {
- boolean resendOp = false;
- boolean removeOwnerFromStream = false;
- SocketAddress previousAddr = addr;
- String reason = cause.getMessage();
- if (cause instanceof ConnectionFailedException || cause instanceof java.net.ConnectException) {
- routingService.removeHost(addr, cause);
- onServerLeft(addr, sc);
- removeOwnerFromStream = true;
- // redirect the request to other host.
- resendOp = true;
- } else if (cause instanceof ChannelException) {
- // java.net.ConnectException typically means connection is refused remotely
- // no process listening on remote address/port.
- if (cause.getCause() instanceof java.net.ConnectException) {
- routingService.removeHost(addr, cause.getCause());
- onServerLeft(addr);
- reason = cause.getCause().getMessage();
- } else {
- routingService.removeHost(addr, cause);
- reason = cause.getMessage();
- }
- removeOwnerFromStream = true;
- // redirect the request to other host.
- resendOp = true;
- } else if (cause instanceof ServiceTimeoutException) {
- // redirect the request to itself again, which will backoff for a while
- resendOp = true;
- previousAddr = null;
- } else if (cause instanceof WriteException) {
- // redirect the request to other host.
- resendOp = true;
- } else if (cause instanceof ServiceException) {
- // redirect the request to other host.
- clientManager.removeClient(addr, sc);
- resendOp = true;
- } else if (cause instanceof TApplicationException) {
- handleTApplicationException(cause, op, addr, sc);
- } else if (cause instanceof Failure) {
- handleFinagleFailure((Failure) cause, op, addr);
- } else {
- // Default handler
- handleException(cause, op, addr);
- }
-
- if (op.isPresent()) {
- if (removeOwnerFromStream) {
- ownershipCache.removeOwnerFromStream(op.get().stream, addr, reason);
- }
- if (resendOp) {
- doSend(op.get(), previousAddr);
- }
- }
- }
-
- /**
- * Redirect the request to new proxy <i>newAddr</i>. If <i>newAddr</i> is null,
- * it would pick up a host from routing service.
- *
- * @param op
- * stream operation
- * @param newAddr
- * new proxy address
- */
- void redirect(StreamOp op, SocketAddress newAddr) {
- ownershipCache.getOwnershipStatsLogger().onRedirect(op.stream);
- if (null != newAddr) {
- logger.debug("Redirect request {} to new owner {}.", op, newAddr);
- op.send(newAddr);
- } else {
- doSend(op, null);
- }
- }
-
- void handleFinagleFailure(Failure failure,
- Optional<StreamOp> op,
- SocketAddress addr) {
- if (failure.isFlagged(Failure.Restartable())) {
- if (op.isPresent()) {
- // redirect the request to other host
- doSend(op.get(), addr);
- }
- } else {
- // fail the request if it is other types of failures
- handleException(failure, op, addr);
- }
- }
-
- void handleException(Throwable cause,
- Optional<StreamOp> op,
- SocketAddress addr) {
- // RequestTimeoutException: fail it and let client decide whether to retry or not.
-
- // FailedFastException:
- // We don't actually know when FailedFastException will be thrown
- // so properly we just throw it back to application to let application
- // handle it.
-
- // Other Exceptions: as we don't know how to handle them properly so throw them to client
- if (op.isPresent()) {
- logger.error("Failed to write request to {} @ {} : {}",
- new Object[]{op.get().stream, addr, cause.toString()});
- op.get().fail(addr, cause);
- }
- }
-
- void handleTApplicationException(Throwable cause,
- Optional<StreamOp> op,
- SocketAddress addr,
- ProxyClient sc) {
- TApplicationException ex = (TApplicationException) cause;
- if (ex.getType() == TApplicationException.UNKNOWN_METHOD) {
- // if we encountered unknown method exception on thrift server, it means this proxy
- // has problem. we should remove it from routing service, clean up ownerships
- routingService.removeHost(addr, cause);
- onServerLeft(addr, sc);
- if (op.isPresent()) {
- ownershipCache.removeOwnerFromStream(op.get().stream, addr, cause.getMessage());
- doSend(op.get(), addr);
- }
- } else {
- handleException(cause, op, addr);
- }
- }
-
- void handleRedirectResponse(ResponseHeader header, StreamOp op, SocketAddress curAddr) {
- SocketAddress ownerAddr = null;
- if (header.isSetLocation()) {
- String owner = header.getLocation();
- try {
- ownerAddr = DLSocketAddress.deserialize(owner).getSocketAddress();
- // if we are receiving a direct request to same host, we won't try the same host.
- // as the proxy will shut itself down if it redirects client to itself.
- if (curAddr.equals(ownerAddr)) {
- logger.warn("Request to stream {} is redirected to same server {}!", op.stream, curAddr);
- ownerAddr = null;
- } else {
- // update ownership when redirects.
- ownershipCache.updateOwner(op.stream, ownerAddr);
- }
- } catch (IOException e) {
- ownerAddr = null;
- }
- }
- redirect(op, ownerAddr);
- }
-
- void updateOwnership(String stream, String location) {
- try {
- SocketAddress ownerAddr = DLSocketAddress.deserialize(location).getSocketAddress();
- // update ownership
- ownershipCache.updateOwner(stream, ownerAddr);
- } catch (IOException e) {
- logger.warn("Invalid ownership {} found for stream {} : ",
- new Object[] { location, stream, e });
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogMultiStreamWriter.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogMultiStreamWriter.java
deleted file mode 100644
index 8ccbbfc..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogMultiStreamWriter.java
+++ /dev/null
@@ -1,486 +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.client;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.base.Ticker;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordSet;
-import com.twitter.distributedlog.LogRecordSetBuffer;
-import com.twitter.distributedlog.client.speculative.DefaultSpeculativeRequestExecutionPolicy;
-import com.twitter.distributedlog.client.speculative.SpeculativeRequestExecutionPolicy;
-import com.twitter.distributedlog.client.speculative.SpeculativeRequestExecutor;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.finagle.IndividualRequestTimeoutException;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Write to multiple streams.
- */
-public class DistributedLogMultiStreamWriter implements Runnable {
-
- /**
- * Create a new builder to create a multi stream writer.
- *
- * @return a new builder to create a multi stream writer.
- */
- public static Builder newBuilder() {
- return new Builder();
- }
-
- /**
- * Builder for the multi stream writer.
- */
- public static class Builder {
-
- private DistributedLogClient client = null;
- private List<String> streams = null;
- private int bufferSize = 16 * 1024; // 16k
- private long flushIntervalMicros = 2000; // 2ms
- private CompressionCodec.Type codec = CompressionCodec.Type.NONE;
- private ScheduledExecutorService executorService = null;
- private long requestTimeoutMs = 500; // 500ms
- private int firstSpeculativeTimeoutMs = 50; // 50ms
- private int maxSpeculativeTimeoutMs = 200; // 200ms
- private float speculativeBackoffMultiplier = 2;
- private Ticker ticker = Ticker.systemTicker();
-
- private Builder() {}
-
- /**
- * Set the distributedlog client used for multi stream writer.
- *
- * @param client
- * distributedlog client
- * @return builder
- */
- public Builder client(DistributedLogClient client) {
- this.client = client;
- return this;
- }
-
- /**
- * Set the list of streams to write to.
- *
- * @param streams
- * list of streams to write
- * @return builder
- */
- public Builder streams(List<String> streams) {
- this.streams = streams;
- return this;
- }
-
- /**
- * Set the output buffer size.
- *
- * <p>If output buffer size is 0, the writes will be transmitted to
- * wire immediately.
- *
- * @param bufferSize
- * output buffer size
- * @return builder
- */
- public Builder bufferSize(int bufferSize) {
- this.bufferSize = bufferSize;
- return this;
- }
-
- /**
- * Set the flush interval in milliseconds.
- *
- * @param flushIntervalMs
- * flush interval in milliseconds.
- * @return builder
- */
- public Builder flushIntervalMs(int flushIntervalMs) {
- this.flushIntervalMicros = TimeUnit.MILLISECONDS.toMicros(flushIntervalMs);
- return this;
- }
-
- /**
- * Set the flush interval in microseconds.
- *
- * @param flushIntervalMicros
- * flush interval in microseconds.
- * @return builder
- */
- public Builder flushIntervalMicros(int flushIntervalMicros) {
- this.flushIntervalMicros = flushIntervalMicros;
- return this;
- }
-
- /**
- * Set compression codec.
- *
- * @param codec compression codec.
- * @return builder
- */
- public Builder compressionCodec(CompressionCodec.Type codec) {
- this.codec = codec;
- return this;
- }
-
- /**
- * Set the scheduler to flush output buffers.
- *
- * @param executorService
- * executor service to flush output buffers.
- * @return builder
- */
- public Builder scheduler(ScheduledExecutorService executorService) {
- this.executorService = executorService;
- return this;
- }
-
- /**
- * Set request timeout in milliseconds.
- *
- * @param requestTimeoutMs
- * request timeout in milliseconds.
- * @return builder
- */
- public Builder requestTimeoutMs(long requestTimeoutMs) {
- this.requestTimeoutMs = requestTimeoutMs;
- return this;
- }
-
- /**
- * Set the first speculative timeout in milliseconds.
- *
- * <p>The multi-streams writer does speculative writes on streams.
- * The write issues first write request to a stream, if the write request
- * doesn't respond within speculative timeout. it issues next write request
- * to a different stream. It does such speculative retries until receive
- * a success or request timeout ({@link #requestTimeoutMs(long)}).
- *
- * <p>This setting is to configure the first speculative timeout, in milliseconds.
- *
- * @param timeoutMs
- * timeout in milliseconds
- * @return builder
- */
- public Builder firstSpeculativeTimeoutMs(int timeoutMs) {
- this.firstSpeculativeTimeoutMs = timeoutMs;
- return this;
- }
-
- /**
- * Set the max speculative timeout in milliseconds.
- *
- * <p>The multi-streams writer does speculative writes on streams.
- * The write issues first write request to a stream, if the write request
- * doesn't respond within speculative timeout. it issues next write request
- * to a different stream. It does such speculative retries until receive
- * a success or request timeout ({@link #requestTimeoutMs(long)}).
- *
- * <p>This setting is to configure the max speculative timeout, in milliseconds.
- *
- * @param timeoutMs
- * timeout in milliseconds
- * @return builder
- */
- public Builder maxSpeculativeTimeoutMs(int timeoutMs) {
- this.maxSpeculativeTimeoutMs = timeoutMs;
- return this;
- }
-
- /**
- * Set the speculative timeout backoff multiplier.
- *
- * <p>The multi-streams writer does speculative writes on streams.
- * The write issues first write request to a stream, if the write request
- * doesn't respond within speculative timeout. it issues next write request
- * to a different stream. It does such speculative retries until receive
- * a success or request timeout ({@link #requestTimeoutMs(long)}).
- *
- * <p>This setting is to configure the speculative timeout backoff multiplier.
- *
- * @param multiplier
- * backoff multiplier
- * @return builder
- */
- public Builder speculativeBackoffMultiplier(float multiplier) {
- this.speculativeBackoffMultiplier = multiplier;
- return this;
- }
-
- /**
- * Ticker for timing.
- *
- * @param ticker
- * ticker
- * @return builder
- * @see Ticker
- */
- public Builder clockTicker(Ticker ticker) {
- this.ticker = ticker;
- return this;
- }
-
- /**
- * Build the multi stream writer.
- *
- * @return the multi stream writer.
- */
- public DistributedLogMultiStreamWriter build() {
- checkArgument((null != streams && !streams.isEmpty()),
- "No streams provided");
- checkNotNull(client,
- "No distributedlog client provided");
- checkNotNull(codec,
- "No compression codec provided");
- checkArgument(firstSpeculativeTimeoutMs > 0
- && firstSpeculativeTimeoutMs <= maxSpeculativeTimeoutMs
- && speculativeBackoffMultiplier > 0
- && maxSpeculativeTimeoutMs < requestTimeoutMs,
- "Invalid speculative timeout settings");
- return new DistributedLogMultiStreamWriter(
- streams,
- client,
- Math.min(bufferSize, MAX_LOGRECORDSET_SIZE),
- flushIntervalMicros,
- requestTimeoutMs,
- firstSpeculativeTimeoutMs,
- maxSpeculativeTimeoutMs,
- speculativeBackoffMultiplier,
- codec,
- ticker,
- executorService);
- }
- }
-
- /**
- * Pending Write Request.
- */
- class PendingWriteRequest implements FutureEventListener<DLSN>,
- SpeculativeRequestExecutor {
-
- private final LogRecordSetBuffer recordSet;
- private AtomicBoolean complete = new AtomicBoolean(false);
- private final Stopwatch stopwatch = Stopwatch.createStarted(clockTicker);
- private int nextStream;
- private int numTriedStreams = 0;
-
- PendingWriteRequest(LogRecordSetBuffer recordSet) {
- this.recordSet = recordSet;
- this.nextStream = Math.abs(nextStreamId.incrementAndGet()) % numStreams;
- }
-
- synchronized String sendNextWrite() {
- long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
- if (elapsedMs > requestTimeoutMs || numTriedStreams >= numStreams) {
- fail(new IndividualRequestTimeoutException(Duration.fromMilliseconds(elapsedMs)));
- return null;
- }
- try {
- return sendWriteToStream(nextStream);
- } finally {
- nextStream = (nextStream + 1) % numStreams;
- ++numTriedStreams;
- }
- }
-
- synchronized String sendWriteToStream(int streamId) {
- String stream = getStream(streamId);
- client.writeRecordSet(stream, recordSet)
- .addEventListener(this);
- return stream;
- }
-
- @Override
- public void onSuccess(DLSN dlsn) {
- if (!complete.compareAndSet(false, true)) {
- return;
- }
- recordSet.completeTransmit(
- dlsn.getLogSegmentSequenceNo(),
- dlsn.getEntryId(),
- dlsn.getSlotId());
- }
-
- @Override
- public void onFailure(Throwable cause) {
- sendNextWrite();
- }
-
- private void fail(Throwable cause) {
- if (!complete.compareAndSet(false, true)) {
- return;
- }
- recordSet.abortTransmit(cause);
- }
-
- @Override
- public Future<Boolean> issueSpeculativeRequest() {
- return Future.value(!complete.get() && null != sendNextWrite());
- }
- }
-
- private final int numStreams;
- private final List<String> streams;
- private final DistributedLogClient client;
- private final int bufferSize;
- private final long requestTimeoutMs;
- private final SpeculativeRequestExecutionPolicy speculativePolicy;
- private final Ticker clockTicker;
- private final CompressionCodec.Type codec;
- private final ScheduledExecutorService scheduler;
- private final boolean ownScheduler;
- private final AtomicInteger nextStreamId;
- private LogRecordSet.Writer recordSetWriter;
-
- private DistributedLogMultiStreamWriter(List<String> streams,
- DistributedLogClient client,
- int bufferSize,
- long flushIntervalMicros,
- long requestTimeoutMs,
- int firstSpecultiveTimeoutMs,
- int maxSpeculativeTimeoutMs,
- float speculativeBackoffMultiplier,
- CompressionCodec.Type codec,
- Ticker clockTicker,
- ScheduledExecutorService scheduler) {
- this.streams = Lists.newArrayList(streams);
- this.numStreams = this.streams.size();
- this.client = client;
- this.bufferSize = bufferSize;
- this.requestTimeoutMs = requestTimeoutMs;
- this.codec = codec;
- this.clockTicker = clockTicker;
- if (null == scheduler) {
- this.scheduler = Executors.newSingleThreadScheduledExecutor(
- new ThreadFactoryBuilder()
- .setDaemon(true)
- .setNameFormat("MultiStreamWriterFlushThread-%d")
- .build());
- this.ownScheduler = true;
- } else {
- this.scheduler = scheduler;
- this.ownScheduler = false;
- }
- this.speculativePolicy = new DefaultSpeculativeRequestExecutionPolicy(
- firstSpecultiveTimeoutMs,
- maxSpeculativeTimeoutMs,
- speculativeBackoffMultiplier);
- // shuffle the streams
- Collections.shuffle(this.streams);
- this.nextStreamId = new AtomicInteger(0);
- this.recordSetWriter = newRecordSetWriter();
-
- if (flushIntervalMicros > 0) {
- this.scheduler.scheduleAtFixedRate(
- this,
- flushIntervalMicros,
- flushIntervalMicros,
- TimeUnit.MICROSECONDS);
- }
- }
-
- String getStream(int streamId) {
- return streams.get(streamId);
- }
-
- synchronized LogRecordSet.Writer getLogRecordSetWriter() {
- return recordSetWriter;
- }
-
- private LogRecordSet.Writer newRecordSetWriter() {
- return LogRecordSet.newWriter(
- bufferSize,
- codec);
- }
-
- public synchronized Future<DLSN> write(ByteBuffer buffer) {
- int logRecordSize = buffer.remaining();
- if (logRecordSize > MAX_LOGRECORD_SIZE) {
- return Future.exception(new LogRecordTooLongException(
- "Log record of size " + logRecordSize + " written when only "
- + MAX_LOGRECORD_SIZE + " is allowed"));
- }
- // if exceed max number of bytes
- if ((recordSetWriter.getNumBytes() + logRecordSize) > MAX_LOGRECORDSET_SIZE) {
- flush();
- }
- Promise<DLSN> writePromise = new Promise<DLSN>();
- try {
- recordSetWriter.writeRecord(buffer, writePromise);
- } catch (LogRecordTooLongException e) {
- return Future.exception(e);
- } catch (WriteException e) {
- recordSetWriter.abortTransmit(e);
- recordSetWriter = newRecordSetWriter();
- return Future.exception(e);
- }
- if (recordSetWriter.getNumBytes() >= bufferSize) {
- flush();
- }
- return writePromise;
- }
-
- @Override
- public void run() {
- flush();
- }
-
- private void flush() {
- LogRecordSet.Writer recordSetToFlush;
- synchronized (this) {
- if (recordSetWriter.getNumRecords() == 0) {
- return;
- }
- recordSetToFlush = recordSetWriter;
- recordSetWriter = newRecordSetWriter();
- }
- transmit(recordSetToFlush);
- }
-
- private void transmit(LogRecordSet.Writer recordSetToFlush) {
- PendingWriteRequest writeRequest =
- new PendingWriteRequest(recordSetToFlush);
- this.speculativePolicy.initiateSpeculativeRequest(scheduler, writeRequest);
- }
-
- public void close() {
- if (ownScheduler) {
- this.scheduler.shutdown();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/MonitorServiceClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/MonitorServiceClient.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/MonitorServiceClient.java
deleted file mode 100644
index e541578..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/MonitorServiceClient.java
+++ /dev/null
@@ -1,68 +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.client.monitor;
-
-import com.twitter.util.Future;
-import java.net.SocketAddress;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Interface for distributedlog monitor service.
- */
-public interface MonitorServiceClient {
-
- /**
- * Check a given stream.
- *
- * @param stream
- * stream.
- * @return check result.
- */
- Future<Void> check(String stream);
-
- /**
- * Send heartbeat to the stream and its readers.
- *
- * @param stream
- * stream.
- * @return check result.
- */
- Future<Void> heartbeat(String stream);
-
- /**
- * Get current ownership distribution from current monitor service view.
- *
- * @return current ownership distribution
- */
- Map<SocketAddress, Set<String>> getStreamOwnershipDistribution();
-
- /**
- * Enable/Disable accepting new stream on a given proxy.
- *
- * @param enabled
- * flag to enable/disable accepting new streams on a given proxy
- * @return void
- */
- Future<Void> setAcceptNewStream(boolean enabled);
-
- /**
- * Close the client.
- */
- void close();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/package-info.java
deleted file mode 100644
index c4e7df0..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Monitor Client.
- */
-package com.twitter.distributedlog.client.monitor;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/OwnershipCache.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/OwnershipCache.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/OwnershipCache.java
deleted file mode 100644
index 387d727..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/OwnershipCache.java
+++ /dev/null
@@ -1,235 +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.client.ownership;
-
-import com.google.common.collect.ImmutableMap;
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.stats.OwnershipStatsLogger;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timeout;
-import org.jboss.netty.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Client Side Ownership Cache.
- */
-public class OwnershipCache implements TimerTask {
-
- private static final Logger logger = LoggerFactory.getLogger(OwnershipCache.class);
-
- private final ConcurrentHashMap<String, SocketAddress> stream2Addresses =
- new ConcurrentHashMap<String, SocketAddress>();
- private final ConcurrentHashMap<SocketAddress, Set<String>> address2Streams =
- new ConcurrentHashMap<SocketAddress, Set<String>>();
- private final ClientConfig clientConfig;
- private final HashedWheelTimer timer;
-
- // Stats
- private final OwnershipStatsLogger ownershipStatsLogger;
-
- public OwnershipCache(ClientConfig clientConfig,
- HashedWheelTimer timer,
- StatsReceiver statsReceiver,
- StatsReceiver streamStatsReceiver) {
- this.clientConfig = clientConfig;
- this.timer = timer;
- this.ownershipStatsLogger = new OwnershipStatsLogger(statsReceiver, streamStatsReceiver);
- scheduleDumpOwnershipCache();
- }
-
- private void scheduleDumpOwnershipCache() {
- if (clientConfig.isPeriodicDumpOwnershipCacheEnabled()
- && clientConfig.getPeriodicDumpOwnershipCacheIntervalMs() > 0) {
- timer.newTimeout(this, clientConfig.getPeriodicDumpOwnershipCacheIntervalMs(),
- TimeUnit.MILLISECONDS);
- }
- }
-
- @Override
- public void run(Timeout timeout) throws Exception {
- if (timeout.isCancelled()) {
- return;
- }
- logger.info("Ownership cache : {} streams cached, {} hosts cached",
- stream2Addresses.size(), address2Streams.size());
- logger.info("Cached streams : {}", stream2Addresses);
- scheduleDumpOwnershipCache();
- }
-
- public OwnershipStatsLogger getOwnershipStatsLogger() {
- return ownershipStatsLogger;
- }
-
- /**
- * Update ownership of <i>stream</i> to <i>addr</i>.
- *
- * @param stream
- * Stream Name.
- * @param addr
- * Owner Address.
- * @return true if owner is updated
- */
- public boolean updateOwner(String stream, SocketAddress addr) {
- // update ownership
- SocketAddress oldAddr = stream2Addresses.putIfAbsent(stream, addr);
- if (null != oldAddr && oldAddr.equals(addr)) {
- return true;
- }
- if (null != oldAddr) {
- if (stream2Addresses.replace(stream, oldAddr, addr)) {
- // Store the relevant mappings for this topic and host combination
- logger.info("Storing ownership for stream : {}, old host : {}, new host : {}.",
- new Object[] { stream, oldAddr, addr });
- StringBuilder sb = new StringBuilder();
- sb.append("Ownership changed '")
- .append(oldAddr).append("' -> '").append(addr).append("'");
- removeOwnerFromStream(stream, oldAddr, sb.toString());
-
- // update stats
- ownershipStatsLogger.onRemove(stream);
- ownershipStatsLogger.onAdd(stream);
- } else {
- logger.warn("Ownership of stream : {} has been changed from {} to {} when storing host : {}.",
- new Object[] { stream, oldAddr, stream2Addresses.get(stream), addr });
- return false;
- }
- } else {
- logger.info("Storing ownership for stream : {}, host : {}.", stream, addr);
- // update stats
- ownershipStatsLogger.onAdd(stream);
- }
-
- Set<String> streamsForHost = address2Streams.get(addr);
- if (null == streamsForHost) {
- Set<String> newStreamsForHost = new HashSet<String>();
- streamsForHost = address2Streams.putIfAbsent(addr, newStreamsForHost);
- if (null == streamsForHost) {
- streamsForHost = newStreamsForHost;
- }
- }
- synchronized (streamsForHost) {
- // check whether the ownership changed, since it might happend after replace succeed
- if (addr.equals(stream2Addresses.get(stream))) {
- streamsForHost.add(stream);
- }
- }
- return true;
- }
-
- /**
- * Get the cached owner for stream <code>stream</code>.
- *
- * @param stream
- * stream to lookup ownership
- * @return owner's address
- */
- public SocketAddress getOwner(String stream) {
- SocketAddress address = stream2Addresses.get(stream);
- if (null == address) {
- ownershipStatsLogger.onMiss(stream);
- } else {
- ownershipStatsLogger.onHit(stream);
- }
- return address;
- }
-
- /**
- * Remove the owner <code>addr</code> from <code>stream</code> for a given <code>reason</code>.
- *
- * @param stream stream name
- * @param addr owner address
- * @param reason reason to remove ownership
- */
- public void removeOwnerFromStream(String stream, SocketAddress addr, String reason) {
- if (stream2Addresses.remove(stream, addr)) {
- logger.info("Removed stream to host mapping for (stream: {} -> host: {}) : reason = '{}'.",
- new Object[] { stream, addr, reason });
- }
- Set<String> streamsForHost = address2Streams.get(addr);
- if (null != streamsForHost) {
- synchronized (streamsForHost) {
- if (streamsForHost.remove(stream)) {
- logger.info("Removed stream ({}) from host {} : reason = '{}'.",
- new Object[] { stream, addr, reason });
- if (streamsForHost.isEmpty()) {
- address2Streams.remove(addr, streamsForHost);
- }
- ownershipStatsLogger.onRemove(stream);
- }
- }
- }
- }
-
- /**
- * Remove all streams from host <code>addr</code>.
- *
- * @param addr
- * host to remove ownerships
- */
- public void removeAllStreamsFromOwner(SocketAddress addr) {
- logger.info("Remove streams mapping for host {}", addr);
- Set<String> streamsForHost = address2Streams.get(addr);
- if (null != streamsForHost) {
- synchronized (streamsForHost) {
- for (String s : streamsForHost) {
- if (stream2Addresses.remove(s, addr)) {
- logger.info("Removing mapping for stream : {} from host : {}", s, addr);
- ownershipStatsLogger.onRemove(s);
- }
- }
- address2Streams.remove(addr, streamsForHost);
- }
- }
- }
-
- /**
- * Get the number cached streams.
- *
- * @return number cached streams.
- */
- public int getNumCachedStreams() {
- return stream2Addresses.size();
- }
-
- /**
- * Get the stream ownership distribution across proxies.
- *
- * @return stream ownership distribution
- */
- public Map<SocketAddress, Set<String>> getStreamOwnershipDistribution() {
- return ImmutableMap.copyOf(address2Streams);
- }
-
- /**
- * Get the stream ownership mapping.
- *
- * @return stream ownership mapping.
- */
- public Map<String, SocketAddress> getStreamOwnerMapping() {
- return stream2Addresses;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/package-info.java
deleted file mode 100644
index 721702e..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Utils for managing ownership at client side.
- */
-package com.twitter.distributedlog.client.ownership;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/package-info.java
deleted file mode 100644
index aa167fb..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Client.
- */
-package com.twitter.distributedlog.client;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ClusterClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ClusterClient.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ClusterClient.java
deleted file mode 100644
index f8bdae7..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ClusterClient.java
+++ /dev/null
@@ -1,51 +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.client.proxy;
-
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.finagle.Service;
-import com.twitter.finagle.thrift.ThriftClientRequest;
-import com.twitter.util.Future;
-import scala.runtime.BoxedUnit;
-
-/**
- * Cluster client.
- */
-public class ClusterClient {
-
- private final Service<ThriftClientRequest, byte[]> client;
- private final DistributedLogService.ServiceIface service;
-
- public ClusterClient(Service<ThriftClientRequest, byte[]> client,
- DistributedLogService.ServiceIface service) {
- this.client = client;
- this.service = service;
- }
-
- public Service<ThriftClientRequest, byte[]> getClient() {
- return client;
- }
-
- public DistributedLogService.ServiceIface getService() {
- return service;
- }
-
- public Future<BoxedUnit> close() {
- return client.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/HostProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/HostProvider.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/HostProvider.java
deleted file mode 100644
index 4878c1c..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/HostProvider.java
+++ /dev/null
@@ -1,35 +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.client.proxy;
-
-import java.net.SocketAddress;
-import java.util.Set;
-
-/**
- * Provider to provider list of hosts for handshaking.
- */
-public interface HostProvider {
-
- /**
- * Get the list of hosts for handshaking.
- *
- * @return list of hosts for handshaking.
- */
- Set<SocketAddress> getHosts();
-
-}
[41/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
new file mode 100644
index 0000000..f1da33c
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
@@ -0,0 +1,119 @@
+/**
+ * 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 org.apache.distributedlog.client.speculative;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Default implementation of {@link SpeculativeRequestExecutionPolicy}.
+ */
+public class DefaultSpeculativeRequestExecutionPolicy implements SpeculativeRequestExecutionPolicy {
+
+ private static final Logger LOG = LoggerFactory.getLogger(DefaultSpeculativeRequestExecutionPolicy.class);
+ final int firstSpeculativeRequestTimeout;
+ final int maxSpeculativeRequestTimeout;
+ final float backoffMultiplier;
+ int nextSpeculativeRequestTimeout;
+
+ public DefaultSpeculativeRequestExecutionPolicy(int firstSpeculativeRequestTimeout,
+ int maxSpeculativeRequestTimeout,
+ float backoffMultiplier) {
+ this.firstSpeculativeRequestTimeout = firstSpeculativeRequestTimeout;
+ this.maxSpeculativeRequestTimeout = maxSpeculativeRequestTimeout;
+ this.backoffMultiplier = backoffMultiplier;
+ this.nextSpeculativeRequestTimeout = firstSpeculativeRequestTimeout;
+
+ if (backoffMultiplier <= 0) {
+ throw new IllegalArgumentException("Invalid value provided for backoffMultiplier");
+ }
+
+ // Prevent potential over flow
+ if (Math.round((double) maxSpeculativeRequestTimeout * (double) backoffMultiplier) > Integer.MAX_VALUE) {
+ throw new IllegalArgumentException("Invalid values for maxSpeculativeRequestTimeout and backoffMultiplier");
+ }
+ }
+
+ @VisibleForTesting
+ int getNextSpeculativeRequestTimeout() {
+ return nextSpeculativeRequestTimeout;
+ }
+
+ /**
+ * Initialize the speculative request execution policy.
+ *
+ * @param scheduler The scheduler service to issue the speculative request
+ * @param requestExecutor The executor is used to issue the actual speculative requests
+ */
+ @Override
+ public void initiateSpeculativeRequest(final ScheduledExecutorService scheduler,
+ final SpeculativeRequestExecutor requestExecutor) {
+ issueSpeculativeRequest(scheduler, requestExecutor);
+ }
+
+ private void issueSpeculativeRequest(final ScheduledExecutorService scheduler,
+ final SpeculativeRequestExecutor requestExecutor) {
+ Future<Boolean> issueNextRequest = requestExecutor.issueSpeculativeRequest();
+ issueNextRequest.addEventListener(new FutureEventListener<Boolean>() {
+ // we want this handler to run immediately after we push the big red button!
+ @Override
+ public void onSuccess(Boolean issueNextRequest) {
+ if (issueNextRequest) {
+ scheduleSpeculativeRequest(scheduler, requestExecutor, nextSpeculativeRequestTimeout);
+ nextSpeculativeRequestTimeout = Math.min(maxSpeculativeRequestTimeout,
+ (int) (nextSpeculativeRequestTimeout * backoffMultiplier));
+ } else {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Stopped issuing speculative requests for {}, "
+ + "speculativeReadTimeout = {}", requestExecutor, nextSpeculativeRequestTimeout);
+ }
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable thrown) {
+ LOG.warn("Failed to issue speculative request for {}, speculativeReadTimeout = {} : ",
+ new Object[] { requestExecutor, nextSpeculativeRequestTimeout, thrown });
+ }
+ });
+ }
+
+ private void scheduleSpeculativeRequest(final ScheduledExecutorService scheduler,
+ final SpeculativeRequestExecutor requestExecutor,
+ final int speculativeRequestTimeout) {
+ try {
+ scheduler.schedule(new Runnable() {
+ @Override
+ public void run() {
+ issueSpeculativeRequest(scheduler, requestExecutor);
+ }
+ }, speculativeRequestTimeout, TimeUnit.MILLISECONDS);
+ } catch (RejectedExecutionException re) {
+ if (!scheduler.isShutdown()) {
+ LOG.warn("Failed to schedule speculative request for {}, speculativeReadTimeout = {} : ",
+ new Object[]{requestExecutor, speculativeRequestTimeout, re});
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
new file mode 100644
index 0000000..faf45c2
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
@@ -0,0 +1,34 @@
+/**
+ * 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 org.apache.distributedlog.client.speculative;
+
+import java.util.concurrent.ScheduledExecutorService;
+
+/**
+ * Speculative request execution policy.
+ */
+public interface SpeculativeRequestExecutionPolicy {
+ /**
+ * Initialize the speculative request execution policy and initiate requests.
+ *
+ * @param scheduler The scheduler service to issue the speculative request
+ * @param requestExecutor The executor is used to issue the actual speculative requests
+ */
+ void initiateSpeculativeRequest(ScheduledExecutorService scheduler,
+ SpeculativeRequestExecutor requestExecutor);
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java
new file mode 100644
index 0000000..68fe8b0
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java
@@ -0,0 +1,33 @@
+/**
+ * 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 org.apache.distributedlog.client.speculative;
+
+import com.twitter.util.Future;
+
+/**
+ * Executor to execute speculative requests.
+ */
+public interface SpeculativeRequestExecutor {
+
+ /**
+ * Issues a speculative request and indicates if more speculative requests should be issued.
+ *
+ * @return whether more speculative requests should be issued.
+ */
+ Future<Boolean> issueSpeculativeRequest();
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java
new file mode 100644
index 0000000..4bdd4b1
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Speculative Mechanism.
+ */
+package org.apache.distributedlog.client.speculative;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java
new file mode 100644
index 0000000..c2dcddd
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java
@@ -0,0 +1,108 @@
+/**
+ * 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 org.apache.distributedlog.client.stats;
+
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Client Stats.
+ */
+public class ClientStats {
+
+ // Region Resolver
+ private final RegionResolver regionResolver;
+
+ // Stats
+ private final StatsReceiver statsReceiver;
+ private final ClientStatsLogger clientStatsLogger;
+ private final boolean enableRegionStats;
+ private final ConcurrentMap<String, ClientStatsLogger> regionClientStatsLoggers;
+ private final ConcurrentMap<String, OpStats> opStatsMap;
+
+ public ClientStats(StatsReceiver statsReceiver,
+ boolean enableRegionStats,
+ RegionResolver regionResolver) {
+ this.statsReceiver = statsReceiver;
+ this.clientStatsLogger = new ClientStatsLogger(statsReceiver);
+ this.enableRegionStats = enableRegionStats;
+ this.regionClientStatsLoggers = new ConcurrentHashMap<String, ClientStatsLogger>();
+ this.regionResolver = regionResolver;
+ this.opStatsMap = new ConcurrentHashMap<String, OpStats>();
+ }
+
+ public OpStats getOpStats(String op) {
+ OpStats opStats = opStatsMap.get(op);
+ if (null != opStats) {
+ return opStats;
+ }
+ OpStats newStats = new OpStats(statsReceiver.scope(op),
+ enableRegionStats, regionResolver);
+ OpStats oldStats = opStatsMap.putIfAbsent(op, newStats);
+ if (null == oldStats) {
+ return newStats;
+ } else {
+ return oldStats;
+ }
+ }
+
+ private ClientStatsLogger getRegionClientStatsLogger(SocketAddress address) {
+ String region = regionResolver.resolveRegion(address);
+ return getRegionClientStatsLogger(region);
+ }
+
+ private ClientStatsLogger getRegionClientStatsLogger(String region) {
+ ClientStatsLogger statsLogger = regionClientStatsLoggers.get(region);
+ if (null == statsLogger) {
+ ClientStatsLogger newStatsLogger = new ClientStatsLogger(statsReceiver.scope(region));
+ ClientStatsLogger oldStatsLogger = regionClientStatsLoggers.putIfAbsent(region, newStatsLogger);
+ if (null == oldStatsLogger) {
+ statsLogger = newStatsLogger;
+ } else {
+ statsLogger = oldStatsLogger;
+ }
+ }
+ return statsLogger;
+ }
+
+ public StatsReceiver getFinagleStatsReceiver(SocketAddress addr) {
+ if (enableRegionStats && null != addr) {
+ return getRegionClientStatsLogger(addr).getStatsReceiver();
+ } else {
+ return clientStatsLogger.getStatsReceiver();
+ }
+ }
+
+ public void completeProxyRequest(SocketAddress addr, StatusCode code, long startTimeNanos) {
+ clientStatsLogger.completeProxyRequest(code, startTimeNanos);
+ if (enableRegionStats && null != addr) {
+ getRegionClientStatsLogger(addr).completeProxyRequest(code, startTimeNanos);
+ }
+ }
+
+ public void failProxyRequest(SocketAddress addr, Throwable cause, long startTimeNanos) {
+ clientStatsLogger.failProxyRequest(cause, startTimeNanos);
+ if (enableRegionStats && null != addr) {
+ getRegionClientStatsLogger(addr).failProxyRequest(cause, startTimeNanos);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java
new file mode 100644
index 0000000..530c632
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.distributedlog.client.stats;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+import com.twitter.finagle.stats.Counter;
+import com.twitter.finagle.stats.Stat;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Stats Logger to collect client stats.
+ */
+public class ClientStatsLogger {
+
+ // Stats
+ private final StatsReceiver statsReceiver;
+ private final StatsReceiver responseStatsReceiver;
+ private final ConcurrentMap<StatusCode, Counter> responseStats =
+ new ConcurrentHashMap<StatusCode, Counter>();
+ private final StatsReceiver exceptionStatsReceiver;
+ private final ConcurrentMap<Class<?>, Counter> exceptionStats =
+ new ConcurrentHashMap<Class<?>, Counter>();
+
+ private final Stat proxySuccessLatencyStat;
+ private final Stat proxyFailureLatencyStat;
+
+ public ClientStatsLogger(StatsReceiver statsReceiver) {
+ this.statsReceiver = statsReceiver;
+ responseStatsReceiver = statsReceiver.scope("responses");
+ exceptionStatsReceiver = statsReceiver.scope("exceptions");
+ StatsReceiver proxyLatencyStatReceiver = statsReceiver.scope("proxy_request_latency");
+ proxySuccessLatencyStat = proxyLatencyStatReceiver.stat0("success");
+ proxyFailureLatencyStat = proxyLatencyStatReceiver.stat0("failure");
+ }
+
+ public StatsReceiver getStatsReceiver() {
+ return statsReceiver;
+ }
+
+ private Counter getResponseCounter(StatusCode code) {
+ Counter counter = responseStats.get(code);
+ if (null == counter) {
+ Counter newCounter = responseStatsReceiver.counter0(code.name());
+ Counter oldCounter = responseStats.putIfAbsent(code, newCounter);
+ counter = null != oldCounter ? oldCounter : newCounter;
+ }
+ return counter;
+ }
+
+ private Counter getExceptionCounter(Class<?> cls) {
+ Counter counter = exceptionStats.get(cls);
+ if (null == counter) {
+ Counter newCounter = exceptionStatsReceiver.counter0(cls.getName());
+ Counter oldCounter = exceptionStats.putIfAbsent(cls, newCounter);
+ counter = null != oldCounter ? oldCounter : newCounter;
+ }
+ return counter;
+ }
+
+ public void completeProxyRequest(StatusCode code, long startTimeNanos) {
+ getResponseCounter(code).incr();
+ proxySuccessLatencyStat.add(elapsedMicroSec(startTimeNanos));
+ }
+
+ public void failProxyRequest(Throwable cause, long startTimeNanos) {
+ getExceptionCounter(cause.getClass()).incr();
+ proxyFailureLatencyStat.add(elapsedMicroSec(startTimeNanos));
+ }
+
+ static long elapsedMicroSec(long startNanoTime) {
+ return TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startNanoTime);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java
new file mode 100644
index 0000000..7a49faa
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java
@@ -0,0 +1,82 @@
+/**
+ * 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 org.apache.distributedlog.client.stats;
+
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Op Stats.
+ */
+public class OpStats {
+
+ // Region Resolver
+ private final RegionResolver regionResolver;
+
+ // Stats
+ private final StatsReceiver statsReceiver;
+ private final OpStatsLogger opStatsLogger;
+ private final boolean enableRegionStats;
+ private final ConcurrentMap<String, OpStatsLogger> regionOpStatsLoggers;
+
+ public OpStats(StatsReceiver statsReceiver,
+ boolean enableRegionStats,
+ RegionResolver regionResolver) {
+ this.statsReceiver = statsReceiver;
+ this.opStatsLogger = new OpStatsLogger(statsReceiver);
+ this.enableRegionStats = enableRegionStats;
+ this.regionOpStatsLoggers = new ConcurrentHashMap<String, OpStatsLogger>();
+ this.regionResolver = regionResolver;
+ }
+
+ private OpStatsLogger getRegionOpStatsLogger(SocketAddress address) {
+ String region = regionResolver.resolveRegion(address);
+ return getRegionOpStatsLogger(region);
+ }
+
+ private OpStatsLogger getRegionOpStatsLogger(String region) {
+ OpStatsLogger statsLogger = regionOpStatsLoggers.get(region);
+ if (null == statsLogger) {
+ OpStatsLogger newStatsLogger = new OpStatsLogger(statsReceiver.scope(region));
+ OpStatsLogger oldStatsLogger = regionOpStatsLoggers.putIfAbsent(region, newStatsLogger);
+ if (null == oldStatsLogger) {
+ statsLogger = newStatsLogger;
+ } else {
+ statsLogger = oldStatsLogger;
+ }
+ }
+ return statsLogger;
+ }
+
+ public void completeRequest(SocketAddress addr, long micros, int numTries) {
+ opStatsLogger.completeRequest(micros, numTries);
+ if (enableRegionStats && null != addr) {
+ getRegionOpStatsLogger(addr).completeRequest(micros, numTries);
+ }
+ }
+
+ public void failRequest(SocketAddress addr, long micros, int numTries) {
+ opStatsLogger.failRequest(micros, numTries);
+ if (enableRegionStats && null != addr) {
+ getRegionOpStatsLogger(addr).failRequest(micros, numTries);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java
new file mode 100644
index 0000000..b94b4be
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java
@@ -0,0 +1,50 @@
+/**
+ * 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 org.apache.distributedlog.client.stats;
+
+import com.twitter.finagle.stats.Stat;
+import com.twitter.finagle.stats.StatsReceiver;
+
+/**
+ * Stats Logger per operation type.
+ */
+public class OpStatsLogger {
+
+ private final Stat successLatencyStat;
+ private final Stat failureLatencyStat;
+ private final Stat redirectStat;
+
+ public OpStatsLogger(StatsReceiver statsReceiver) {
+ StatsReceiver latencyStatReceiver = statsReceiver.scope("latency");
+ successLatencyStat = latencyStatReceiver.stat0("success");
+ failureLatencyStat = latencyStatReceiver.stat0("failure");
+ StatsReceiver redirectStatReceiver = statsReceiver.scope("redirects");
+ redirectStat = redirectStatReceiver.stat0("times");
+ }
+
+ public void completeRequest(long micros, int numTries) {
+ successLatencyStat.add(micros);
+ redirectStat.add(numTries);
+ }
+
+ public void failRequest(long micros, int numTries) {
+ failureLatencyStat.add(micros);
+ redirectStat.add(numTries);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java
new file mode 100644
index 0000000..110e99a
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java
@@ -0,0 +1,115 @@
+/**
+ * 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 org.apache.distributedlog.client.stats;
+
+import com.twitter.finagle.stats.Counter;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Stats Logger for ownerships.
+ */
+public class OwnershipStatsLogger {
+
+ /**
+ * Ownership related stats.
+ */
+ public static class OwnershipStat {
+ private final Counter hits;
+ private final Counter misses;
+ private final Counter removes;
+ private final Counter redirects;
+ private final Counter adds;
+
+ OwnershipStat(StatsReceiver ownershipStats) {
+ hits = ownershipStats.counter0("hits");
+ misses = ownershipStats.counter0("misses");
+ adds = ownershipStats.counter0("adds");
+ removes = ownershipStats.counter0("removes");
+ redirects = ownershipStats.counter0("redirects");
+ }
+
+ public void onHit() {
+ hits.incr();
+ }
+
+ public void onMiss() {
+ misses.incr();
+ }
+
+ public void onAdd() {
+ adds.incr();
+ }
+
+ public void onRemove() {
+ removes.incr();
+ }
+
+ public void onRedirect() {
+ redirects.incr();
+ }
+
+ }
+
+ private final OwnershipStat ownershipStat;
+ private final StatsReceiver ownershipStatsReceiver;
+ private final ConcurrentMap<String, OwnershipStat> ownershipStats =
+ new ConcurrentHashMap<String, OwnershipStat>();
+
+ public OwnershipStatsLogger(StatsReceiver statsReceiver,
+ StatsReceiver streamStatsReceiver) {
+ this.ownershipStat = new OwnershipStat(statsReceiver.scope("ownership"));
+ this.ownershipStatsReceiver = streamStatsReceiver.scope("perstream_ownership");
+ }
+
+ private OwnershipStat getOwnershipStat(String stream) {
+ OwnershipStat stat = ownershipStats.get(stream);
+ if (null == stat) {
+ OwnershipStat newStat = new OwnershipStat(ownershipStatsReceiver.scope(stream));
+ OwnershipStat oldStat = ownershipStats.putIfAbsent(stream, newStat);
+ stat = null != oldStat ? oldStat : newStat;
+ }
+ return stat;
+ }
+
+ public void onMiss(String stream) {
+ ownershipStat.onMiss();
+ getOwnershipStat(stream).onMiss();
+ }
+
+ public void onHit(String stream) {
+ ownershipStat.onHit();
+ getOwnershipStat(stream).onHit();
+ }
+
+ public void onRedirect(String stream) {
+ ownershipStat.onRedirect();
+ getOwnershipStat(stream).onRedirect();
+ }
+
+ public void onRemove(String stream) {
+ ownershipStat.onRemove();
+ getOwnershipStat(stream).onRemove();
+ }
+
+ public void onAdd(String stream) {
+ ownershipStat.onAdd();
+ getOwnershipStat(stream).onAdd();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java
new file mode 100644
index 0000000..106d3fc
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Client side stats utils.
+ */
+package org.apache.distributedlog.client.stats;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java
new file mode 100644
index 0000000..68e6825
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java
@@ -0,0 +1,161 @@
+/**
+ * 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 org.apache.distributedlog.service;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+
+/**
+ * Socket Address identifier for a DL proxy.
+ */
+public class DLSocketAddress {
+
+ private static final int VERSION = 1;
+
+ private static final String COLON = ":";
+ private static final String SEP = ";";
+
+ private final int shard;
+ private final InetSocketAddress socketAddress;
+
+ public DLSocketAddress(int shard, InetSocketAddress socketAddress) {
+ this.shard = shard;
+ this.socketAddress = socketAddress;
+ }
+
+ /**
+ * Shard id for dl write proxy.
+ *
+ * @return shard id for dl write proxy.
+ */
+ public int getShard() {
+ return shard;
+ }
+
+ /**
+ * Socket address for dl write proxy.
+ *
+ * @return socket address for dl write proxy
+ */
+ public InetSocketAddress getSocketAddress() {
+ return socketAddress;
+ }
+
+ /**
+ * Serialize the write proxy identifier to string.
+ *
+ * @return serialized write proxy identifier.
+ */
+ public String serialize() {
+ return toLockId(socketAddress, shard);
+ }
+
+ @Override
+ public int hashCode() {
+ return socketAddress.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof DLSocketAddress)) {
+ return false;
+ }
+ DLSocketAddress other = (DLSocketAddress) obj;
+ return shard == other.shard && socketAddress.equals(other.socketAddress);
+ }
+
+ @Override
+ public String toString() {
+ return toLockId(socketAddress, shard);
+ }
+
+ /**
+ * Deserialize proxy address from a string representation.
+ *
+ * @param lockId
+ * string representation of the proxy address.
+ * @return proxy address.
+ * @throws IOException
+ */
+ public static DLSocketAddress deserialize(String lockId) throws IOException {
+ String parts[] = lockId.split(SEP);
+ if (3 != parts.length) {
+ throw new IOException("Invalid dl socket address " + lockId);
+ }
+ int version;
+ try {
+ version = Integer.parseInt(parts[0]);
+ } catch (NumberFormatException nfe) {
+ throw new IOException("Invalid version found in " + lockId, nfe);
+ }
+ if (VERSION != version) {
+ throw new IOException("Invalid version " + version + " found in " + lockId + ", expected " + VERSION);
+ }
+ int shardId;
+ try {
+ shardId = Integer.parseInt(parts[1]);
+ } catch (NumberFormatException nfe) {
+ throw new IOException("Invalid shard id found in " + lockId, nfe);
+ }
+ InetSocketAddress address = parseSocketAddress(parts[2]);
+ return new DLSocketAddress(shardId, address);
+ }
+
+ /**
+ * Parse the inet socket address from the string representation.
+ *
+ * @param addr
+ * string representation
+ * @return inet socket address
+ */
+ public static InetSocketAddress parseSocketAddress(String addr) {
+ String[] parts = addr.split(COLON);
+ checkArgument(parts.length == 2);
+ String hostname = parts[0];
+ int port = Integer.parseInt(parts[1]);
+ return new InetSocketAddress(hostname, port);
+ }
+
+ public static InetSocketAddress getSocketAddress(int port) throws UnknownHostException {
+ return new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), port);
+ }
+
+ /**
+ * Convert inet socket address to the string representation.
+ *
+ * @param address
+ * inet socket address.
+ * @return string representation of inet socket address.
+ */
+ public static String toString(InetSocketAddress address) {
+ StringBuilder sb = new StringBuilder();
+ sb.append(address.getHostName()).append(COLON).append(address.getPort());
+ return sb.toString();
+ }
+
+ public static String toLockId(InetSocketAddress address, int shard) {
+ StringBuilder sb = new StringBuilder();
+ sb.append(VERSION).append(SEP).append(shard).append(SEP).append(toString(address));
+ return sb.toString();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java
new file mode 100644
index 0000000..9f30815
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java
@@ -0,0 +1,108 @@
+/**
+ * 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 org.apache.distributedlog.service;
+
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecordSetBuffer;
+import com.twitter.util.Future;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Interface for distributedlog client.
+ */
+public interface DistributedLogClient {
+ /**
+ * Write <i>data</i> to a given <i>stream</i>.
+ *
+ * @param stream
+ * Stream Name.
+ * @param data
+ * Data to write.
+ * @return a future representing a sequence id returned for this write.
+ */
+ Future<DLSN> write(String stream, ByteBuffer data);
+
+ /**
+ * Write record set to a given <i>stream</i>.
+ *
+ * <p>The record set is built from {@link org.apache.distributedlog.LogRecordSet.Writer}
+ *
+ * @param stream stream to write to
+ * @param recordSet record set
+ */
+ Future<DLSN> writeRecordSet(String stream, LogRecordSetBuffer recordSet);
+
+ /**
+ * Write <i>data</i> in bulk to a given <i>stream</i>.
+ *
+ * <p>Return a list of Future dlsns, one for each submitted buffer. In the event of a partial
+ * failure--ex. some specific buffer write fails, all subsequent writes
+ * will also fail.
+ *
+ * @param stream
+ * Stream Name.
+ * @param data
+ * Data to write.
+ * @return a list of futures, one for each submitted buffer.
+ */
+ List<Future<DLSN>> writeBulk(String stream, List<ByteBuffer> data);
+
+ /**
+ * Truncate the stream to a given <i>dlsn</i>.
+ *
+ * @param stream
+ * Stream Name.
+ * @param dlsn
+ * DLSN to truncate until.
+ * @return a future representing the truncation.
+ */
+ Future<Boolean> truncate(String stream, DLSN dlsn);
+
+ /**
+ * Release the ownership of a stream <i>stream</i>.
+ *
+ * @param stream
+ * Stream Name to release.
+ * @return a future representing the release operation.
+ */
+ Future<Void> release(String stream);
+
+ /**
+ * Delete a given stream <i>stream</i>.
+ *
+ * @param stream
+ * Stream Name to delete.
+ * @return a future representing the delete operation.
+ */
+ Future<Void> delete(String stream);
+
+ /**
+ * Create a stream with name <i>stream</i>.
+ *
+ * @param stream
+ * Stream Name to create.
+ * @return a future representing the create operation.
+ */
+ Future<Void> create(String stream);
+
+ /**
+ * Close the client.
+ */
+ void close();
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java
new file mode 100644
index 0000000..0e2a152
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java
@@ -0,0 +1,608 @@
+/**
+ * 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 org.apache.distributedlog.service;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.DistributedLogClientImpl;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.client.proxy.ClusterClient;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.client.routing.RegionsRoutingService;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.client.routing.RoutingUtils;
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import com.twitter.finagle.Name;
+import com.twitter.finagle.Resolver$;
+import com.twitter.finagle.Service;
+import com.twitter.finagle.ThriftMux;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId;
+import com.twitter.finagle.thrift.ThriftClientFramedCodec;
+import com.twitter.finagle.thrift.ThriftClientRequest;
+import com.twitter.util.Duration;
+import java.net.SocketAddress;
+import java.net.URI;
+import java.util.Random;
+import org.apache.commons.lang.StringUtils;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * Builder to build {@link DistributedLogClient}.
+ */
+public final class DistributedLogClientBuilder {
+
+ private static final Logger logger = LoggerFactory.getLogger(DistributedLogClientBuilder.class);
+
+ private static final Random random = new Random(System.currentTimeMillis());
+
+ private String name = null;
+ private ClientId clientId = null;
+ private RoutingService.Builder routingServiceBuilder = null;
+ private ClientBuilder clientBuilder = null;
+ private String serverRoutingServiceFinagleName = null;
+ private StatsReceiver statsReceiver = new NullStatsReceiver();
+ private StatsReceiver streamStatsReceiver = new NullStatsReceiver();
+ private ClientConfig clientConfig = new ClientConfig();
+ private boolean enableRegionStats = false;
+ private final RegionResolver regionResolver = new DefaultRegionResolver();
+
+ /**
+ * Create a client builder.
+ *
+ * @return client builder
+ */
+ public static DistributedLogClientBuilder newBuilder() {
+ return new DistributedLogClientBuilder();
+ }
+
+ /**
+ * Create a new client builder from an existing {@code builder}.
+ *
+ * @param builder the existing builder.
+ * @return a new client builder.
+ */
+ public static DistributedLogClientBuilder newBuilder(DistributedLogClientBuilder builder) {
+ DistributedLogClientBuilder newBuilder = new DistributedLogClientBuilder();
+ newBuilder.name = builder.name;
+ newBuilder.clientId = builder.clientId;
+ newBuilder.clientBuilder = builder.clientBuilder;
+ newBuilder.routingServiceBuilder = builder.routingServiceBuilder;
+ newBuilder.statsReceiver = builder.statsReceiver;
+ newBuilder.streamStatsReceiver = builder.streamStatsReceiver;
+ newBuilder.enableRegionStats = builder.enableRegionStats;
+ newBuilder.serverRoutingServiceFinagleName = builder.serverRoutingServiceFinagleName;
+ newBuilder.clientConfig = ClientConfig.newConfig(builder.clientConfig);
+ return newBuilder;
+ }
+
+ // private constructor
+ private DistributedLogClientBuilder() {}
+
+ /**
+ * Client Name.
+ *
+ * @param name
+ * client name
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder name(String name) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.name = name;
+ return newBuilder;
+ }
+
+ /**
+ * Client ID.
+ *
+ * @param clientId
+ * client id
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder clientId(ClientId clientId) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientId = clientId;
+ return newBuilder;
+ }
+
+ /**
+ * Serverset to access proxy services.
+ *
+ * @param serverSet
+ * server set.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder serverSet(ServerSet serverSet) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(serverSet);
+ newBuilder.enableRegionStats = false;
+ return newBuilder;
+ }
+
+ /**
+ * Server Sets to access proxy services.
+ *
+ * <p>The <i>local</i> server set will be tried first then <i>remotes</i>.
+ *
+ * @param local local server set.
+ * @param remotes remote server sets.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder serverSets(ServerSet local, ServerSet...remotes) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ RoutingService.Builder[] builders = new RoutingService.Builder[remotes.length + 1];
+ builders[0] = RoutingUtils.buildRoutingService(local);
+ for (int i = 1; i < builders.length; i++) {
+ builders[i] = RoutingUtils.buildRoutingService(remotes[i - 1]);
+ }
+ newBuilder.routingServiceBuilder = RegionsRoutingService.newBuilder()
+ .resolver(regionResolver)
+ .routingServiceBuilders(builders);
+ newBuilder.enableRegionStats = remotes.length > 0;
+ return newBuilder;
+ }
+
+ /**
+ * Name to access proxy services.
+ *
+ * @param finagleNameStr
+ * finagle name string.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder finagleNameStr(String finagleNameStr) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(finagleNameStr);
+ newBuilder.enableRegionStats = false;
+ return newBuilder;
+ }
+
+ /**
+ * Finagle name strs to access proxy services.
+ *
+ * <p>The <i>local</i> finalge name str will be tried first, then <i>remotes</i>.
+ *
+ * @param local local server set.
+ * @param remotes remote server sets.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder finagleNameStrs(String local, String...remotes) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ RoutingService.Builder[] builders = new RoutingService.Builder[remotes.length + 1];
+ builders[0] = RoutingUtils.buildRoutingService(local);
+ for (int i = 1; i < builders.length; i++) {
+ builders[i] = RoutingUtils.buildRoutingService(remotes[i - 1]);
+ }
+ newBuilder.routingServiceBuilder = RegionsRoutingService.newBuilder()
+ .routingServiceBuilders(builders)
+ .resolver(regionResolver);
+ newBuilder.enableRegionStats = remotes.length > 0;
+ return newBuilder;
+ }
+
+ /**
+ * URI to access proxy services.
+ *
+ * <p>Assuming the write proxies are announced under `.write_proxy` of the provided namespace uri.
+ * The builder will convert the dl uri (e.g. distributedlog://{zkserver}/path/to/namespace) to
+ * zookeeper serverset based finagle name str (`zk!{zkserver}!/path/to/namespace/.write_proxy`)
+ *
+ * @param uri namespace uri to access the serverset of write proxies
+ * @return distributedlog builder
+ */
+ public DistributedLogClientBuilder uri(URI uri) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ String zkServers = uri.getAuthority().replace(";", ",");
+ String[] zkServerList = StringUtils.split(zkServers, ',');
+ String finagleNameStr = String.format(
+ "zk!%s!%s/.write_proxy",
+ zkServerList[random.nextInt(zkServerList.length)], // zk server
+ uri.getPath());
+ newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(finagleNameStr);
+ newBuilder.enableRegionStats = false;
+ return newBuilder;
+ }
+
+ /**
+ * Address of write proxy to connect.
+ *
+ * @param address
+ * write proxy address.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder host(SocketAddress address) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(address);
+ newBuilder.enableRegionStats = false;
+ return newBuilder;
+ }
+
+ private DistributedLogClientBuilder routingServiceBuilder(RoutingService.Builder builder) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.routingServiceBuilder = builder;
+ newBuilder.enableRegionStats = false;
+ return newBuilder;
+ }
+
+ /**
+ * Routing Service to access proxy services.
+ *
+ * @param routingService
+ * routing service
+ * @return client builder.
+ */
+ @VisibleForTesting
+ public DistributedLogClientBuilder routingService(RoutingService routingService) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(routingService);
+ newBuilder.enableRegionStats = false;
+ return newBuilder;
+ }
+
+ /**
+ * Stats receiver to expose client stats.
+ *
+ * @param statsReceiver
+ * stats receiver.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder statsReceiver(StatsReceiver statsReceiver) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.statsReceiver = statsReceiver;
+ return newBuilder;
+ }
+
+ /**
+ * Stream Stats Receiver to expose per stream stats.
+ *
+ * @param streamStatsReceiver
+ * stream stats receiver
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder streamStatsReceiver(StatsReceiver streamStatsReceiver) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.streamStatsReceiver = streamStatsReceiver;
+ return newBuilder;
+ }
+
+ /**
+ * Set underlying finagle client builder.
+ *
+ * @param builder
+ * finagle client builder.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder clientBuilder(ClientBuilder builder) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientBuilder = builder;
+ return newBuilder;
+ }
+
+ /**
+ * Backoff time when redirecting to an already retried host.
+ *
+ * @param ms
+ * backoff time.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder redirectBackoffStartMs(int ms) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setRedirectBackoffStartMs(ms);
+ return newBuilder;
+ }
+
+ /**
+ * Max backoff time when redirecting to an already retried host.
+ *
+ * @param ms
+ * backoff time.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder redirectBackoffMaxMs(int ms) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setRedirectBackoffMaxMs(ms);
+ return newBuilder;
+ }
+
+ /**
+ * Max redirects that is allowed per request.
+ *
+ * <p>If <i>redirects</i> are exhausted, fail the request immediately.
+ *
+ * @param redirects
+ * max redirects allowed before failing a request.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder maxRedirects(int redirects) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setMaxRedirects(redirects);
+ return newBuilder;
+ }
+
+ /**
+ * Timeout per request in millis.
+ *
+ * @param timeoutMs
+ * timeout per request in millis.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder requestTimeoutMs(int timeoutMs) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setRequestTimeoutMs(timeoutMs);
+ return newBuilder;
+ }
+
+ /**
+ * Set thriftmux enabled.
+ *
+ * @param enabled
+ * is thriftmux enabled
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder thriftmux(boolean enabled) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setThriftMux(enabled);
+ return newBuilder;
+ }
+
+ /**
+ * Set failfast stream exception handling enabled.
+ *
+ * @param enabled
+ * is failfast exception handling enabled
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder streamFailfast(boolean enabled) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setStreamFailfast(enabled);
+ return newBuilder;
+ }
+
+ /**
+ * Set the regex to match stream names that the client cares about.
+ *
+ * @param nameRegex
+ * stream name regex
+ * @return client builder
+ */
+ public DistributedLogClientBuilder streamNameRegex(String nameRegex) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setStreamNameRegex(nameRegex);
+ return newBuilder;
+ }
+
+ /**
+ * Whether to use the new handshake endpoint to exchange ownership cache.
+ *
+ * <p>Enable this when the servers are updated to support handshaking with client info.
+ *
+ * @param enabled
+ * new handshake endpoint is enabled.
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder handshakeWithClientInfo(boolean enabled) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setHandshakeWithClientInfo(enabled);
+ return newBuilder;
+ }
+
+ /**
+ * Set the periodic handshake interval in milliseconds.
+ *
+ * <p>Every <code>intervalMs</code>, the DL client will handshake with existing proxies again.
+ * If the interval is less than ownership sync interval, the handshake won't sync ownerships. Otherwise, it will.
+ *
+ * @see #periodicOwnershipSyncIntervalMs(long)
+ * @param intervalMs
+ * handshake interval
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder periodicHandshakeIntervalMs(long intervalMs) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setPeriodicHandshakeIntervalMs(intervalMs);
+ return newBuilder;
+ }
+
+ /**
+ * Set the periodic ownership sync interval in milliseconds.
+ *
+ * <p>If periodic handshake is enabled, the handshake will sync ownership if the elapsed time is larger than
+ * sync interval.
+ *
+ * @see #periodicHandshakeIntervalMs(long)
+ * @param intervalMs
+ * interval that handshake should sync ownerships.
+ * @return client builder
+ */
+ public DistributedLogClientBuilder periodicOwnershipSyncIntervalMs(long intervalMs) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setPeriodicOwnershipSyncIntervalMs(intervalMs);
+ return newBuilder;
+ }
+
+ /**
+ * Enable/Disable periodic dumping ownership cache.
+ *
+ * @param enabled
+ * flag to enable/disable periodic dumping ownership cache
+ * @return client builder.
+ */
+ public DistributedLogClientBuilder periodicDumpOwnershipCache(boolean enabled) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setPeriodicDumpOwnershipCacheEnabled(enabled);
+ return newBuilder;
+ }
+
+ /**
+ * Set periodic dumping ownership cache interval.
+ *
+ * @param intervalMs
+ * interval on dumping ownership cache, in millis.
+ * @return client builder
+ */
+ public DistributedLogClientBuilder periodicDumpOwnershipCacheIntervalMs(long intervalMs) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setPeriodicDumpOwnershipCacheIntervalMs(intervalMs);
+ return newBuilder;
+ }
+
+ /**
+ * Enable handshake tracing.
+ *
+ * @param enabled
+ * flag to enable/disable handshake tracing
+ * @return client builder
+ */
+ public DistributedLogClientBuilder handshakeTracing(boolean enabled) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setHandshakeTracingEnabled(enabled);
+ return newBuilder;
+ }
+
+ /**
+ * Enable checksum on requests to the proxy.
+ *
+ * @param enabled
+ * flag to enable/disable checksum
+ * @return client builder
+ */
+ public DistributedLogClientBuilder checksum(boolean enabled) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig.setChecksumEnabled(enabled);
+ return newBuilder;
+ }
+
+ /**
+ * Configure the finagle name string for the server-side routing service.
+ *
+ * @param nameStr name string of the server-side routing service
+ * @return client builder
+ */
+ public DistributedLogClientBuilder serverRoutingServiceFinagleNameStr(String nameStr) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.serverRoutingServiceFinagleName = nameStr;
+ return newBuilder;
+ }
+
+ DistributedLogClientBuilder clientConfig(ClientConfig clientConfig) {
+ DistributedLogClientBuilder newBuilder = newBuilder(this);
+ newBuilder.clientConfig = ClientConfig.newConfig(clientConfig);
+ return newBuilder;
+ }
+
+ /**
+ * Build distributedlog client.
+ *
+ * @return distributedlog client.
+ */
+ public DistributedLogClient build() {
+ return buildClient();
+ }
+
+ /**
+ * Build monitor service client.
+ *
+ * @return monitor service client.
+ */
+ public MonitorServiceClient buildMonitorClient() {
+
+ return buildClient();
+ }
+
+ @SuppressWarnings("unchecked")
+ ClusterClient buildServerRoutingServiceClient(String serverRoutingServiceFinagleName) {
+ ClientBuilder builder = this.clientBuilder;
+ if (null == builder) {
+ builder = ClientBuilder.get()
+ .tcpConnectTimeout(Duration.fromMilliseconds(200))
+ .connectTimeout(Duration.fromMilliseconds(200))
+ .requestTimeout(Duration.fromSeconds(1))
+ .retries(20);
+ if (!clientConfig.getThriftMux()) {
+ builder = builder.hostConnectionLimit(1);
+ }
+ }
+ if (clientConfig.getThriftMux()) {
+ builder = builder.stack(ThriftMux.client().withClientId(clientId));
+ } else {
+ builder = builder.codec(ThriftClientFramedCodec.apply(Option.apply(clientId)));
+ }
+
+ Name name;
+ try {
+ name = Resolver$.MODULE$.eval(serverRoutingServiceFinagleName);
+ } catch (Exception exc) {
+ logger.error("Exception in Resolver.eval for name {}", serverRoutingServiceFinagleName, exc);
+ throw new RuntimeException(exc);
+ }
+
+ // builder the client
+ Service<ThriftClientRequest, byte[]> client =
+ ClientBuilder.safeBuildFactory(
+ builder.dest(name).reportTo(statsReceiver.scope("routing"))
+ ).toService();
+ DistributedLogService.ServiceIface service =
+ new DistributedLogService.ServiceToClient(client, new TBinaryProtocol.Factory());
+ return new ClusterClient(client, service);
+ }
+
+ DistributedLogClientImpl buildClient() {
+ checkNotNull(name, "No name provided.");
+ checkNotNull(clientId, "No client id provided.");
+ checkNotNull(routingServiceBuilder, "No routing service builder provided.");
+ checkNotNull(statsReceiver, "No stats receiver provided.");
+ if (null == streamStatsReceiver) {
+ streamStatsReceiver = new NullStatsReceiver();
+ }
+
+ Optional<ClusterClient> serverRoutingServiceClient = Optional.absent();
+ if (null != serverRoutingServiceFinagleName) {
+ serverRoutingServiceClient = Optional.of(
+ buildServerRoutingServiceClient(serverRoutingServiceFinagleName));
+ }
+
+ RoutingService routingService = routingServiceBuilder
+ .statsReceiver(statsReceiver.scope("routing"))
+ .build();
+ DistributedLogClientImpl clientImpl =
+ new DistributedLogClientImpl(
+ name,
+ clientId,
+ routingService,
+ clientBuilder,
+ clientConfig,
+ serverRoutingServiceClient,
+ statsReceiver,
+ streamStatsReceiver,
+ regionResolver,
+ enableRegionStats);
+ routingService.startService();
+ clientImpl.handshake();
+ return clientImpl;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/service/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/service/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/service/package-info.java
new file mode 100644
index 0000000..033882f
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/service/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * DistributedLog Service Client.
+ */
+package org.apache.distributedlog.service;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/resources/findbugsExclude.xml
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/resources/findbugsExclude.xml b/distributedlog-client/src/main/resources/findbugsExclude.xml
index 29e1a16..05ee085 100644
--- a/distributedlog-client/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-client/src/main/resources/findbugsExclude.xml
@@ -18,6 +18,6 @@
<FindBugsFilter>
<Match>
<!-- generated code, we can't be held responsible for findbugs in it //-->
- <Class name="~com\.twitter\.distributedlog\.thrift.*" />
+ <Class name="~org\.apache\.distributedlog\.thrift.*" />
</Match>
</FindBugsFilter>
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/TestDistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/TestDistributedLogMultiStreamWriter.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/TestDistributedLogMultiStreamWriter.java
deleted file mode 100644
index b302439..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/TestDistributedLogMultiStreamWriter.java
+++ /dev/null
@@ -1,383 +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.client;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.LogRecordSet;
-import com.twitter.distributedlog.LogRecordSetBuffer;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.finagle.IndividualRequestTimeoutException;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import java.nio.ByteBuffer;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test {@link DistributedLogMultiStreamWriter}.
- */
-public class TestDistributedLogMultiStreamWriter {
-
- @Test(timeout = 20000, expected = IllegalArgumentException.class)
- public void testBuildWithNullStreams() throws Exception {
- DistributedLogMultiStreamWriter.newBuilder()
- .build();
- }
-
- @Test(timeout = 20000, expected = IllegalArgumentException.class)
- public void testBuildWithEmptyStreamList() throws Exception {
- DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.<String>newArrayList())
- .build();
- }
-
- @Test(timeout = 20000, expected = NullPointerException.class)
- public void testBuildWithNullClient() throws Exception {
- DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .build();
- }
-
- @Test(timeout = 20000, expected = NullPointerException.class)
- public void testBuildWithNullCodec() throws Exception {
- DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(mock(DistributedLogClient.class))
- .compressionCodec(null)
- .build();
- }
-
- @Test(timeout = 20000, expected = IllegalArgumentException.class)
- public void testBuildWithInvalidSpeculativeSettings1()
- throws Exception {
- DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(mock(DistributedLogClient.class))
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(-1)
- .build();
- }
-
- @Test(timeout = 20000, expected = IllegalArgumentException.class)
- public void testBuildWithInvalidSpeculativeSettings2()
- throws Exception {
- DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(mock(DistributedLogClient.class))
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(10)
- .maxSpeculativeTimeoutMs(5)
- .build();
- }
-
- @Test(timeout = 20000, expected = IllegalArgumentException.class)
- public void testBuildWithInvalidSpeculativeSettings3()
- throws Exception {
- DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(mock(DistributedLogClient.class))
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(10)
- .maxSpeculativeTimeoutMs(20)
- .speculativeBackoffMultiplier(-1)
- .build();
- }
-
- @Test(timeout = 20000, expected = IllegalArgumentException.class)
- public void testBuildWithInvalidSpeculativeSettings4()
- throws Exception {
- DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(mock(DistributedLogClient.class))
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(10)
- .maxSpeculativeTimeoutMs(20)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(10)
- .build();
- }
-
- @Test(timeout = 20000)
- public void testBuildMultiStreamWriter()
- throws Exception {
- DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(mock(DistributedLogClient.class))
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(10)
- .maxSpeculativeTimeoutMs(20)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(50)
- .build();
- assertTrue(true);
- }
-
- @Test(timeout = 20000)
- public void testBuildWithPeriodicalFlushEnabled() throws Exception {
- ScheduledExecutorService executorService = mock(ScheduledExecutorService.class);
- DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(mock(DistributedLogClient.class))
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(10)
- .maxSpeculativeTimeoutMs(20)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(50)
- .flushIntervalMs(1000)
- .scheduler(executorService)
- .build();
- verify(executorService, times(1)).scheduleAtFixedRate(writer, 1000000, 1000000, TimeUnit.MICROSECONDS);
- }
-
- @Test(timeout = 20000)
- public void testBuildWithPeriodicalFlushDisabled() throws Exception {
- ScheduledExecutorService executorService = mock(ScheduledExecutorService.class);
- DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(mock(DistributedLogClient.class))
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(10)
- .maxSpeculativeTimeoutMs(20)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(50)
- .flushIntervalMs(0)
- .scheduler(executorService)
- .build();
- verify(executorService, times(0)).scheduleAtFixedRate(writer, 1000, 1000, TimeUnit.MILLISECONDS);
- writer.close();
- }
-
- @Test(timeout = 20000)
- public void testFlushWhenBufferIsFull() throws Exception {
- DistributedLogClient client = mock(DistributedLogClient.class);
- when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
- .thenReturn(Future.value(new DLSN(1L, 1L, 999L)));
-
- ScheduledExecutorService executorService =
- Executors.newSingleThreadScheduledExecutor();
- DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(client)
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(100000)
- .maxSpeculativeTimeoutMs(200000)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(500000)
- .flushIntervalMs(0)
- .bufferSize(0)
- .scheduler(executorService)
- .build();
-
- ByteBuffer buffer = ByteBuffer.wrap("test".getBytes(UTF_8));
- writer.write(buffer);
-
- verify(client, times(1)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
-
- writer.close();
- }
-
- @Test(timeout = 20000)
- public void testFlushWhenExceedMaxLogRecordSetSize()
- throws Exception {
- DistributedLogClient client = mock(DistributedLogClient.class);
- when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
- .thenReturn(Future.value(new DLSN(1L, 1L, 999L)));
- ScheduledExecutorService executorService =
- Executors.newSingleThreadScheduledExecutor();
- DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(client)
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(100000)
- .maxSpeculativeTimeoutMs(200000)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(500000)
- .flushIntervalMs(0)
- .bufferSize(Integer.MAX_VALUE)
- .scheduler(executorService)
- .build();
-
- byte[] data = new byte[LogRecord.MAX_LOGRECORD_SIZE - 3 * 100];
- ByteBuffer buffer1 = ByteBuffer.wrap(data);
- writer.write(buffer1);
- verify(client, times(0)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
- LogRecordSet.Writer recordSetWriter1 = writer.getLogRecordSetWriter();
- assertEquals(1, recordSetWriter1.getNumRecords());
- assertEquals(LogRecordSet.HEADER_LEN + 4 + data.length, recordSetWriter1.getNumBytes());
-
- ByteBuffer buffer2 = ByteBuffer.wrap(data);
- writer.write(buffer2);
- verify(client, times(1)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
- LogRecordSet.Writer recordSetWriter2 = writer.getLogRecordSetWriter();
- assertEquals(1, recordSetWriter2.getNumRecords());
- assertEquals(LogRecordSet.HEADER_LEN + 4 + data.length, recordSetWriter2.getNumBytes());
- assertTrue(recordSetWriter1 != recordSetWriter2);
-
- writer.close();
- }
-
- @Test(timeout = 20000)
- public void testWriteTooLargeRecord() throws Exception {
- DistributedLogClient client = mock(DistributedLogClient.class);
- DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(client)
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(100000)
- .maxSpeculativeTimeoutMs(200000)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(5000000)
- .flushIntervalMs(0)
- .bufferSize(0)
- .build();
-
- byte[] data = new byte[LogRecord.MAX_LOGRECORD_SIZE + 10];
- ByteBuffer buffer = ByteBuffer.wrap(data);
- Future<DLSN> writeFuture = writer.write(buffer);
- assertTrue(writeFuture.isDefined());
- try {
- Await.result(writeFuture);
- fail("Should fail on writing too long record");
- } catch (LogRecordTooLongException lrtle) {
- // expected
- }
- writer.close();
- }
-
- @Test(timeout = 20000)
- public void testSpeculativeWrite() throws Exception {
- DistributedLogClient client = mock(DistributedLogClient.class);
- DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(client)
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(10)
- .maxSpeculativeTimeoutMs(20)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(5000000)
- .flushIntervalMs(0)
- .bufferSize(0)
- .build();
-
- final String secondStream = writer.getStream(1);
-
- final DLSN dlsn = new DLSN(99L, 88L, 0L);
-
- Mockito.doAnswer(new Answer() {
- @Override
- public Object answer(InvocationOnMock invocation) throws Throwable {
- Object[] arguments = invocation.getArguments();
- String stream = (String) arguments[0];
- if (stream.equals(secondStream)) {
- return Future.value(dlsn);
- } else {
- return new Promise<DLSN>();
- }
- }
- }).when(client).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
-
- byte[] data = "test-test".getBytes(UTF_8);
- ByteBuffer buffer = ByteBuffer.wrap(data);
- Future<DLSN> writeFuture = writer.write(buffer);
- DLSN writeDLSN = Await.result(writeFuture);
- assertEquals(dlsn, writeDLSN);
- writer.close();
- }
-
- @Test(timeout = 20000)
- public void testPeriodicalFlush() throws Exception {
- DistributedLogClient client = mock(DistributedLogClient.class);
- DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(client)
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(10)
- .maxSpeculativeTimeoutMs(20)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(5000000)
- .flushIntervalMs(10)
- .bufferSize(Integer.MAX_VALUE)
- .build();
-
- final DLSN dlsn = new DLSN(99L, 88L, 0L);
-
- Mockito.doAnswer(new Answer() {
- @Override
- public Object answer(InvocationOnMock invocation) throws Throwable {
- return Future.value(dlsn);
- }
- }).when(client).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
-
- byte[] data = "test-test".getBytes(UTF_8);
- ByteBuffer buffer = ByteBuffer.wrap(data);
- Future<DLSN> writeFuture = writer.write(buffer);
- DLSN writeDLSN = Await.result(writeFuture);
- assertEquals(dlsn, writeDLSN);
- writer.close();
- }
-
- @Test(timeout = 20000)
- public void testFailRequestAfterRetriedAllStreams() throws Exception {
- DistributedLogClient client = mock(DistributedLogClient.class);
- when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
- .thenReturn(new Promise<DLSN>());
- DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
- .streams(Lists.newArrayList("stream1", "stream2"))
- .client(client)
- .compressionCodec(CompressionCodec.Type.LZ4)
- .firstSpeculativeTimeoutMs(10)
- .maxSpeculativeTimeoutMs(20)
- .speculativeBackoffMultiplier(2)
- .requestTimeoutMs(5000000)
- .flushIntervalMs(10)
- .bufferSize(Integer.MAX_VALUE)
- .build();
-
- byte[] data = "test-test".getBytes(UTF_8);
- ByteBuffer buffer = ByteBuffer.wrap(data);
- Future<DLSN> writeFuture = writer.write(buffer);
- try {
- Await.result(writeFuture);
- fail("Should fail the request after retries all streams");
- } catch (IndividualRequestTimeoutException e) {
- long timeoutMs = e.timeout().inMilliseconds();
- assertTrue(timeoutMs >= (10 + 20) && timeoutMs < 5000000);
- }
- writer.close();
- }
-}
[28/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
deleted file mode 100644
index 40e3930..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
+++ /dev/null
@@ -1,992 +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.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.google.common.base.Ticker;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.callback.LogSegmentListener;
-import com.twitter.distributedlog.exceptions.AlreadyTruncatedTransactionException;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.EndOfLogSegmentException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryReader;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Function0;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Futures;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Function1;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * New ReadAhead Reader that uses {@link com.twitter.distributedlog.logsegment.LogSegmentEntryReader}.
- *
- * NOTE: all the state changes happen in the same thread. All *unsafe* methods should be submitted to the order
- * scheduler using stream name as the key.
- */
-public class ReadAheadEntryReader implements
- AsyncCloseable,
- LogSegmentListener,
- LogSegmentEntryReader.StateChangeListener,
- FutureEventListener<List<Entry.Reader>> {
-
- private static final Logger logger = LoggerFactory.getLogger(ReadAheadEntryReader.class);
-
- //
- // Static Functions
- //
-
- private static AbstractFunction1<LogSegmentEntryReader, BoxedUnit> START_READER_FUNC = new AbstractFunction1<LogSegmentEntryReader, BoxedUnit>() {
- @Override
- public BoxedUnit apply(LogSegmentEntryReader reader) {
- reader.start();
- return BoxedUnit.UNIT;
- }
- };
-
- //
- // Internal Classes
- //
-
- class SegmentReader implements FutureEventListener<LogSegmentEntryReader> {
-
- private LogSegmentMetadata metadata;
- private final long startEntryId;
- private Future<LogSegmentEntryReader> openFuture = null;
- private LogSegmentEntryReader reader = null;
- private boolean isStarted = false;
- private boolean isClosed = false;
-
- SegmentReader(LogSegmentMetadata metadata,
- long startEntryId) {
- this.metadata = metadata;
- this.startEntryId = startEntryId;
- }
-
- synchronized LogSegmentEntryReader getEntryReader() {
- return reader;
- }
-
- synchronized boolean isBeyondLastAddConfirmed() {
- return null != reader && reader.isBeyondLastAddConfirmed();
- }
-
- synchronized LogSegmentMetadata getSegment() {
- return metadata;
- }
-
- synchronized boolean isReaderOpen() {
- return null != openFuture;
- }
-
- synchronized void openReader() {
- if (null != openFuture) {
- return;
- }
- openFuture = entryStore.openReader(metadata, startEntryId).addEventListener(this);
- }
-
- synchronized boolean isReaderStarted() {
- return isStarted;
- }
-
- synchronized void startRead() {
- if (isStarted) {
- return;
- }
- isStarted = true;
- if (null != reader) {
- reader.start();
- } else {
- openFuture.onSuccess(START_READER_FUNC);
- }
- }
-
- synchronized Future<List<Entry.Reader>> readNext() {
- if (null != reader) {
- checkCatchingUpStatus(reader);
- return reader.readNext(numReadAheadEntries);
- } else {
- return openFuture.flatMap(readFunc);
- }
- }
-
- synchronized void updateLogSegmentMetadata(final LogSegmentMetadata segment) {
- if (null != reader) {
- reader.onLogSegmentMetadataUpdated(segment);
- this.metadata = segment;
- } else {
- openFuture.onSuccess(new AbstractFunction1<LogSegmentEntryReader, BoxedUnit>() {
- @Override
- public BoxedUnit apply(LogSegmentEntryReader reader) {
- reader.onLogSegmentMetadataUpdated(segment);
- synchronized (SegmentReader.this) {
- SegmentReader.this.metadata = segment;
- }
- return BoxedUnit.UNIT;
- }
- });
- }
- }
-
- @Override
- synchronized public void onSuccess(LogSegmentEntryReader reader) {
- this.reader = reader;
- if (reader.getSegment().isInProgress()) {
- reader.registerListener(ReadAheadEntryReader.this);
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- // no-op, the failure will be propagated on first read.
- }
-
- synchronized boolean isClosed() {
- return isClosed;
- }
-
- synchronized Future<Void> close() {
- if (null == openFuture) {
- return Future.Void();
- }
- return openFuture.flatMap(new AbstractFunction1<LogSegmentEntryReader, Future<Void>>() {
- @Override
- public Future<Void> apply(LogSegmentEntryReader reader) {
- return reader.asyncClose();
- }
- }).ensure(new Function0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- synchronized (SegmentReader.this) {
- isClosed = true;
- }
- return null;
- }
- });
- }
- }
-
- private class ReadEntriesFunc extends AbstractFunction1<LogSegmentEntryReader, Future<List<Entry.Reader>>> {
-
- private final int numEntries;
-
- ReadEntriesFunc(int numEntries) {
- this.numEntries = numEntries;
- }
-
- @Override
- public Future<List<Entry.Reader>> apply(LogSegmentEntryReader reader) {
- checkCatchingUpStatus(reader);
- return reader.readNext(numEntries);
- }
- }
-
- private abstract class CloseableRunnable implements Runnable {
-
- @Override
- public void run() {
- synchronized (ReadAheadEntryReader.this) {
- if (null != closePromise) {
- return;
- }
- }
- try {
- safeRun();
- } catch (Throwable cause) {
- logger.error("Caught unexpected exception : ", cause);
- }
- }
-
- abstract void safeRun();
-
- }
-
- //
- // Functions
- //
- private final Function1<LogSegmentEntryReader, Future<List<Entry.Reader>>> readFunc;
- private final Function0<BoxedUnit> removeClosedSegmentReadersFunc = new Function0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- removeClosedSegmentReaders();
- return BoxedUnit.UNIT;
- }
- };
-
- //
- // Resources
- //
- private final DistributedLogConfiguration conf;
- private final BKLogReadHandler readHandler;
- private final LogSegmentEntryStore entryStore;
- private final OrderedScheduler scheduler;
-
- //
- // Parameters
- //
- private final String streamName;
- private final DLSN fromDLSN;
- private final int maxCachedEntries;
- private final int numReadAheadEntries;
- private final int idleWarnThresholdMillis;
-
- //
- // Cache
- //
- private final LinkedBlockingQueue<Entry.Reader> entryQueue;
-
- //
- // State of the reader
- //
-
- private final AtomicBoolean started = new AtomicBoolean(false);
- private boolean isInitialized = false;
- private boolean readAheadPaused = false;
- private Promise<Void> closePromise = null;
- // segment readers
- private long currentSegmentSequenceNumber;
- private SegmentReader currentSegmentReader;
- private SegmentReader nextSegmentReader;
- private DLSN lastDLSN;
- private final EntryPosition nextEntryPosition;
- private volatile boolean isCatchingUp = true;
- private final LinkedList<SegmentReader> segmentReaders;
- private final LinkedList<SegmentReader> segmentReadersToClose;
- // last exception that this reader encounters
- private final AtomicReference<IOException> lastException = new AtomicReference<IOException>(null);
- // last entry added time
- private final Stopwatch lastEntryAddedTime;
- // state change notification
- private final CopyOnWriteArraySet<AsyncNotification> stateChangeNotifications =
- new CopyOnWriteArraySet<AsyncNotification>();
- // idle reader check task
- private final ScheduledFuture<?> idleReaderCheckTask;
-
- //
- // Stats
- //
- private final AlertStatsLogger alertStatsLogger;
-
- public ReadAheadEntryReader(String streamName,
- DLSN fromDLSN,
- DistributedLogConfiguration conf,
- BKLogReadHandler readHandler,
- LogSegmentEntryStore entryStore,
- OrderedScheduler scheduler,
- Ticker ticker,
- AlertStatsLogger alertStatsLogger) {
- this.streamName = streamName;
- this.fromDLSN = lastDLSN = fromDLSN;
- this.nextEntryPosition = new EntryPosition(
- fromDLSN.getLogSegmentSequenceNo(),
- fromDLSN.getEntryId());
- this.conf = conf;
- this.maxCachedEntries = conf.getReadAheadMaxRecords();
- this.numReadAheadEntries = conf.getReadAheadBatchSize();
- this.idleWarnThresholdMillis = conf.getReaderIdleWarnThresholdMillis();
- this.readHandler = readHandler;
- this.entryStore = entryStore;
- this.scheduler = scheduler;
- this.readFunc = new ReadEntriesFunc(numReadAheadEntries);
- this.alertStatsLogger = alertStatsLogger;
-
- // create the segment reader list
- this.segmentReaders = new LinkedList<SegmentReader>();
- this.segmentReadersToClose = new LinkedList<SegmentReader>();
- // create the readahead entry queue
- this.entryQueue = new LinkedBlockingQueue<Entry.Reader>();
-
- // start the idle reader detection
- lastEntryAddedTime = Stopwatch.createStarted(ticker);
- // start the idle reader check task
- idleReaderCheckTask = scheduleIdleReaderTaskIfNecessary();
- }
-
- private ScheduledFuture<?> scheduleIdleReaderTaskIfNecessary() {
- if (idleWarnThresholdMillis < Integer.MAX_VALUE && idleWarnThresholdMillis > 0) {
- return scheduler.scheduleAtFixedRate(streamName, new Runnable() {
- @Override
- public void run() {
- if (!isReaderIdle(idleWarnThresholdMillis, TimeUnit.MILLISECONDS)) {
- return;
- }
- // the readahead has been idle
- unsafeCheckIfReadAheadIsIdle();
- }
- }, idleWarnThresholdMillis, idleWarnThresholdMillis, TimeUnit.MILLISECONDS);
- }
- return null;
- }
-
- private void unsafeCheckIfReadAheadIsIdle() {
- boolean forceReadLogSegments =
- (null == currentSegmentReader) || currentSegmentReader.isBeyondLastAddConfirmed();
- if (forceReadLogSegments) {
- readHandler.readLogSegmentsFromStore(
- LogSegmentMetadata.COMPARATOR,
- LogSegmentFilter.DEFAULT_FILTER,
- null
- ).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
- @Override
- public void onFailure(Throwable cause) {
- // do nothing here since it would be retried on next idle reader check task
- }
-
- @Override
- public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
- onSegmentsUpdated(segments.getValue());
- }
- });
- }
- }
-
- private void cancelIdleReaderTask() {
- if (null != idleReaderCheckTask) {
- idleReaderCheckTask.cancel(true);
- }
- }
-
- @VisibleForTesting
- EntryPosition getNextEntryPosition() {
- return nextEntryPosition;
- }
-
- @VisibleForTesting
- SegmentReader getCurrentSegmentReader() {
- return currentSegmentReader;
- }
-
- @VisibleForTesting
- long getCurrentSegmentSequenceNumber() {
- return currentSegmentSequenceNumber;
- }
-
- @VisibleForTesting
- SegmentReader getNextSegmentReader() {
- return nextSegmentReader;
- }
-
- @VisibleForTesting
- LinkedList<SegmentReader> getSegmentReaders() {
- return segmentReaders;
- }
-
- @VisibleForTesting
- boolean isInitialized() {
- return isInitialized;
- }
-
- private void orderedSubmit(Runnable runnable) {
- synchronized (this) {
- if (null != closePromise) {
- return;
- }
- }
- try {
- scheduler.submit(streamName, runnable);
- } catch (RejectedExecutionException ree) {
- logger.debug("Failed to submit and execute an operation for readhead entry reader of {}",
- streamName, ree);
- }
- }
-
- public void start(final List<LogSegmentMetadata> segmentList) {
- logger.info("Starting the readahead entry reader for {} : segments = {}",
- readHandler.getFullyQualifiedName(), segmentList);
- started.set(true);
- processLogSegments(segmentList);
- }
-
- private void removeClosedSegmentReaders() {
- orderedSubmit(new CloseableRunnable() {
- @Override
- void safeRun() {
- unsafeRemoveClosedSegmentReaders();
- }
- });
- }
-
- private void unsafeRemoveClosedSegmentReaders() {
- SegmentReader reader = segmentReadersToClose.peekFirst();
- while (null != reader) {
- if (reader.isClosed()) {
- segmentReadersToClose.pollFirst();
- reader = segmentReadersToClose.peekFirst();
- } else {
- break;
- }
- }
- }
-
- @Override
- public Future<Void> asyncClose() {
- final Promise<Void> closeFuture;
- synchronized (this) {
- if (null != closePromise) {
- return closePromise;
- }
- closePromise = closeFuture = new Promise<Void>();
- }
-
- // cancel the idle reader task
- cancelIdleReaderTask();
-
- // use runnable here instead of CloseableRunnable,
- // because we need this to be executed
- try {
- scheduler.submit(streamName, new Runnable() {
- @Override
- public void run() {
- unsafeAsyncClose(closeFuture);
- }
- });
- } catch (RejectedExecutionException ree) {
- logger.warn("Scheduler has been shutdown before closing the readahead entry reader for stream {}",
- streamName, ree);
- unsafeAsyncClose(closeFuture);
- }
-
- return closeFuture;
- }
-
- private void unsafeAsyncClose(Promise<Void> closePromise) {
- List<Future<Void>> closeFutures = Lists.newArrayListWithExpectedSize(
- segmentReaders.size() + segmentReadersToClose.size() + 1);
- if (null != currentSegmentReader) {
- segmentReadersToClose.add(currentSegmentReader);
- }
- if (null != nextSegmentReader) {
- segmentReadersToClose.add(nextSegmentReader);
- }
- for (SegmentReader reader : segmentReaders) {
- segmentReadersToClose.add(reader);
- }
- segmentReaders.clear();
- for (SegmentReader reader : segmentReadersToClose) {
- closeFutures.add(reader.close());
- }
- Futures.collect(closeFutures).proxyTo(closePromise);
- }
-
- //
- // Reader State Changes
- //
-
- ReadAheadEntryReader addStateChangeNotification(AsyncNotification notification) {
- this.stateChangeNotifications.add(notification);
- return this;
- }
-
- ReadAheadEntryReader removeStateChangeNotification(AsyncNotification notification) {
- this.stateChangeNotifications.remove(notification);
- return this;
- }
-
- private void notifyStateChangeOnSuccess() {
- for (AsyncNotification notification : stateChangeNotifications) {
- notification.notifyOnOperationComplete();
- }
- }
-
- private void notifyStateChangeOnFailure(Throwable cause) {
- for (AsyncNotification notification : stateChangeNotifications) {
- notification.notifyOnError(cause);
- }
- }
-
- void setLastException(IOException cause) {
- if (!lastException.compareAndSet(null, cause)) {
- logger.debug("last exception has already been set to ", lastException.get());
- }
- // the exception is set and notify the state change
- notifyStateChangeOnFailure(cause);
- }
-
- void checkLastException() throws IOException {
- if (null != lastException.get()) {
- throw lastException.get();
- }
- }
-
- void checkCatchingUpStatus(LogSegmentEntryReader reader) {
- if (reader.getSegment().isInProgress()
- && isCatchingUp
- && reader.hasCaughtUpOnInprogress()) {
- logger.info("ReadAhead for {} is caught up at entry {} @ log segment {}.",
- new Object[] { readHandler.getFullyQualifiedName(),
- reader.getLastAddConfirmed(), reader.getSegment() });
- isCatchingUp = false;
- }
- }
-
- 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
- //
-
- @Override
- public void onSuccess(List<Entry.Reader> entries) {
- lastEntryAddedTime.reset().start();
- for (Entry.Reader entry : entries) {
- entryQueue.add(entry);
- }
- if (!entries.isEmpty()) {
- Entry.Reader lastEntry = entries.get(entries.size() - 1);
- nextEntryPosition.advance(lastEntry.getLSSN(), lastEntry.getEntryId() + 1);
- }
- // notify on data available
- notifyStateChangeOnSuccess();
- if (entryQueue.size() >= maxCachedEntries) {
- pauseReadAheadOnCacheFull();
- } else {
- scheduleReadNext();
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- if (cause instanceof EndOfLogSegmentException) {
- // we reach end of the log segment
- moveToNextLogSegment();
- return;
- }
- if (cause instanceof IOException) {
- setLastException((IOException) cause);
- } else {
- setLastException(new UnexpectedException("Unexpected non I/O exception", cause));
- }
- }
-
- private synchronized void invokeReadAhead() {
- if (readAheadPaused) {
- scheduleReadNext();
- readAheadPaused = false;
- }
- }
-
- private synchronized void pauseReadAheadOnCacheFull() {
- this.readAheadPaused = true;
- if (!isCacheFull()) {
- invokeReadAhead();
- }
- }
-
- private synchronized void pauseReadAheadOnNoMoreLogSegments() {
- this.readAheadPaused = true;
- }
-
- //
- // Cache Related Methods
- //
-
- public Entry.Reader getNextReadAheadEntry(long waitTime, TimeUnit waitTimeUnit) throws IOException {
- if (null != lastException.get()) {
- throw lastException.get();
- }
- Entry.Reader entry;
- try {
- entry = entryQueue.poll(waitTime, waitTimeUnit);
- } catch (InterruptedException e) {
- throw new DLInterruptedException("Interrupted on waiting next readahead entry : ", e);
- }
- try {
- return entry;
- } finally {
- // resume readahead if the cache becomes empty
- if (null != entry && !isCacheFull()) {
- invokeReadAhead();
- }
- }
- }
-
- /**
- * Return number cached entries.
- *
- * @return number cached entries.
- */
- public int getNumCachedEntries() {
- return entryQueue.size();
- }
-
- /**
- * Return if the cache is full.
- *
- * @return true if the cache is full, otherwise false.
- */
- public boolean isCacheFull() {
- return getNumCachedEntries() >= maxCachedEntries;
- }
-
- @VisibleForTesting
- public boolean isCacheEmpty() {
- return entryQueue.isEmpty();
- }
-
- /**
- * Check whether the readahead becomes stall.
- *
- * @param idleReaderErrorThreshold idle reader error threshold
- * @param timeUnit time unit of the idle reader error threshold
- * @return true if the readahead becomes stall, otherwise false.
- */
- public boolean isReaderIdle(int idleReaderErrorThreshold, TimeUnit timeUnit) {
- return (lastEntryAddedTime.elapsed(timeUnit) > idleReaderErrorThreshold);
- }
-
- //
- // LogSegment Management
- //
-
- void processLogSegments(final List<LogSegmentMetadata> segments) {
- orderedSubmit(new CloseableRunnable() {
- @Override
- void safeRun() {
- unsafeProcessLogSegments(segments);
- }
- });
- }
-
- private void unsafeProcessLogSegments(List<LogSegmentMetadata> segments) {
- if (isInitialized) {
- unsafeReinitializeLogSegments(segments);
- } else {
- unsafeInitializeLogSegments(segments);
- }
- }
-
- /**
- * Update the log segment metadata.
- *
- * @param reader the reader to update the metadata
- * @param newMetadata the new metadata received
- * @return true if successfully, false on encountering errors
- */
- private boolean updateLogSegmentMetadata(SegmentReader reader,
- LogSegmentMetadata newMetadata) {
- if (reader.getSegment().getLogSegmentSequenceNumber() != newMetadata.getLogSegmentSequenceNumber()) {
- setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for "
- + streamName + " : current segment = " + reader.getSegment() + ", new segment = " + newMetadata));
- return false;
- }
- if (!reader.getSegment().isInProgress() && newMetadata.isInProgress()) {
- setLastException(new DLIllegalStateException("An inprogress log segment " + newMetadata
- + " received after a closed log segment " + reader.getSegment() + " on reading segment "
- + newMetadata.getLogSegmentSequenceNumber() + " @ stream " + streamName));
- return false;
- }
- if (reader.getSegment().isInProgress() && !newMetadata.isInProgress()) {
- reader.updateLogSegmentMetadata(newMetadata);
- }
- return true;
- }
-
- /**
- * Reinitialize the log segments
- */
- private void unsafeReinitializeLogSegments(List<LogSegmentMetadata> segments) {
- logger.info("Reinitialize log segments with {}", segments);
- int segmentIdx = 0;
- for (; segmentIdx < segments.size(); segmentIdx++) {
- LogSegmentMetadata segment = segments.get(segmentIdx);
- if (segment.getLogSegmentSequenceNumber() < currentSegmentSequenceNumber) {
- continue;
- }
- break;
- }
- if (segmentIdx >= segments.size()) {
- return;
- }
- LogSegmentMetadata segment = segments.get(segmentIdx);
- if (null != currentSegmentReader) {
- if (!updateLogSegmentMetadata(currentSegmentReader, segment)) {
- return;
- }
- } else {
- if (currentSegmentSequenceNumber != segment.getLogSegmentSequenceNumber()) {
- setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for "
- + streamName + " : current segment sn = " + currentSegmentSequenceNumber
- + ", new segment sn = " + segment.getLogSegmentSequenceNumber()));
- return;
- }
- }
- segmentIdx++;
- if (segmentIdx >= segments.size()) {
- return;
- }
- // check next segment
- segment = segments.get(segmentIdx);
- if (null != nextSegmentReader) {
- if (!updateLogSegmentMetadata(nextSegmentReader, segment)) {
- return;
- }
- segmentIdx++;
- }
- // check the segment readers in the queue
- for (int readerIdx = 0;
- readerIdx < segmentReaders.size() && segmentIdx < segments.size();
- readerIdx++, segmentIdx++) {
- SegmentReader reader = segmentReaders.get(readerIdx);
- segment = segments.get(segmentIdx);
- if (!updateLogSegmentMetadata(reader, segment)) {
- return;
- }
- }
- // add the remaining segments to the reader queue
- for (; segmentIdx < segments.size(); segmentIdx++) {
- segment = segments.get(segmentIdx);
- SegmentReader reader = new SegmentReader(segment, 0L);
- reader.openReader();
- segmentReaders.add(reader);
- }
- if (null == currentSegmentReader) {
- unsafeMoveToNextLogSegment();
- }
- // resume readahead if necessary
- invokeReadAhead();
- }
-
- /**
- * Initialize the reader with the log <i>segments</i>.
- *
- * @param segments list of log segments
- */
- private void unsafeInitializeLogSegments(List<LogSegmentMetadata> segments) {
- if (segments.isEmpty()) {
- // not initialize the background reader, until the first log segment is notified
- return;
- }
- boolean skipTruncatedLogSegments = true;
- DLSN dlsnToStart = fromDLSN;
- // positioning the reader
- for (int i = 0; i < segments.size(); i++) {
- LogSegmentMetadata segment = segments.get(i);
- // skip any log segments that have smaller log segment sequence numbers
- if (segment.getLogSegmentSequenceNumber() < fromDLSN.getLogSegmentSequenceNo()) {
- continue;
- }
- // if the log segment is truncated, skip it.
- if (skipTruncatedLogSegments &&
- !conf.getIgnoreTruncationStatus() &&
- segment.isTruncated()) {
- continue;
- }
- // if the log segment is partially truncated, move the start dlsn to the min active dlsn
- if (skipTruncatedLogSegments &&
- !conf.getIgnoreTruncationStatus() &&
- segment.isPartiallyTruncated()) {
- if (segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
- dlsnToStart = segment.getMinActiveDLSN();
- }
- }
- skipTruncatedLogSegments = false;
- if (!isAllowedToPosition(segment, dlsnToStart)) {
- logger.error("segment {} is not allowed to position at {}", segment, dlsnToStart);
- return;
- }
-
- SegmentReader reader = new SegmentReader(segment,
- segment.getLogSegmentSequenceNumber() == dlsnToStart.getLogSegmentSequenceNo()
- ? dlsnToStart.getEntryId() : 0L);
- segmentReaders.add(reader);
- }
- if (segmentReaders.isEmpty()) {
- // not initialize the background reader, until the first log segment is available to read
- return;
- }
- currentSegmentReader = segmentReaders.pollFirst();
- currentSegmentReader.openReader();
- currentSegmentReader.startRead();
- currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
- unsafeReadNext(currentSegmentReader);
- if (!segmentReaders.isEmpty()) {
- for (SegmentReader reader : segmentReaders) {
- reader.openReader();
- }
- unsafePrefetchNextSegment(true);
- }
- // mark the reader initialized
- isInitialized = true;
- }
-
- private void unsafePrefetchNextSegment(boolean onlyInprogressLogSegment) {
- SegmentReader nextReader = segmentReaders.peekFirst();
- // open the next log segment if it is inprogress
- if (null != nextReader) {
- if (onlyInprogressLogSegment && !nextReader.getSegment().isInProgress()) {
- return;
- }
- nextReader.startRead();
- nextSegmentReader = nextReader;
- segmentReaders.pollFirst();
- }
- }
-
- /**
- * Check if we are allowed to position the reader at <i>fromDLSN</i>.
- *
- * @return true if it is allowed, otherwise false.
- */
- private boolean isAllowedToPosition(LogSegmentMetadata segment, DLSN fromDLSN) {
- if (segment.isTruncated()
- && segment.getLastDLSN().compareTo(fromDLSN) >= 0
- && !conf.getIgnoreTruncationStatus()) {
- setLastException(new AlreadyTruncatedTransactionException(streamName
- + " : trying to position read ahead at " + fromDLSN
- + " on a segment " + segment + " that is already marked as truncated"));
- return false;
- }
- if (segment.isPartiallyTruncated() &&
- segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
- if (conf.getAlertWhenPositioningOnTruncated()) {
- alertStatsLogger.raise("Trying to position reader on {} when {} is marked partially truncated",
- fromDLSN, segment);
- }
- if (!conf.getIgnoreTruncationStatus()) {
- logger.error("{}: Trying to position reader on {} when {} is marked partially truncated",
- new Object[]{ streamName, fromDLSN, segment });
-
- setLastException(new AlreadyTruncatedTransactionException(streamName
- + " : trying to position read ahead at " + fromDLSN
- + " on a segment " + segment + " that is already marked as truncated"));
- return false;
- }
- }
- return true;
- }
-
- void moveToNextLogSegment() {
- orderedSubmit(new CloseableRunnable() {
- @Override
- void safeRun() {
- unsafeMoveToNextLogSegment();
- }
- });
- }
-
- private void unsafeMoveToNextLogSegment() {
- if (null != currentSegmentReader) {
- segmentReadersToClose.add(currentSegmentReader);
- currentSegmentReader.close().ensure(removeClosedSegmentReadersFunc);
- logger.debug("close current segment reader {}", currentSegmentReader.getSegment());
- currentSegmentReader = null;
- }
- boolean hasSegmentToRead = false;
- if (null != nextSegmentReader) {
- currentSegmentReader = nextSegmentReader;
- logger.debug("move to read segment {}", currentSegmentReader.getSegment());
- currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
- nextSegmentReader = null;
- // start reading
- unsafeReadNext(currentSegmentReader);
- unsafePrefetchNextSegment(true);
- hasSegmentToRead = true;
- } else {
- unsafePrefetchNextSegment(false);
- if (null != nextSegmentReader) {
- currentSegmentReader = nextSegmentReader;
- logger.debug("move to read segment {}", currentSegmentReader.getSegment());
- currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
- nextSegmentReader = null;
- unsafeReadNext(currentSegmentReader);
- unsafePrefetchNextSegment(true);
- hasSegmentToRead = true;
- }
- }
- if (!hasSegmentToRead) { // no more segment to read, wait until new log segment arrive
- if (isCatchingUp) {
- logger.info("ReadAhead for {} is caught up and no log segments to read now",
- readHandler.getFullyQualifiedName());
- isCatchingUp = false;
- }
- pauseReadAheadOnNoMoreLogSegments();
- }
- }
-
- void scheduleReadNext() {
- orderedSubmit(new CloseableRunnable() {
- @Override
- void safeRun() {
- if (null == currentSegmentReader) {
- pauseReadAheadOnNoMoreLogSegments();
- return;
- }
- unsafeReadNext(currentSegmentReader);
- }
- });
- }
-
- private void unsafeReadNext(SegmentReader reader) {
- reader.readNext().addEventListener(this);
- }
-
- @Override
- public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
- if (!started.get()) {
- return;
- }
- logger.info("segments is updated with {}", segments);
- processLogSegments(segments);
- }
-
- @Override
- public void onLogStreamDeleted() {
- setLastException(new LogNotFoundException("Log stream "
- + streamName + " is deleted"));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadUtils.java
deleted file mode 100644
index f481561..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadUtils.java
+++ /dev/null
@@ -1,782 +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 java.io.IOException;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.selector.FirstDLSNNotLessThanSelector;
-import com.twitter.distributedlog.selector.FirstTxIdNotLessThanSelector;
-import com.twitter.distributedlog.selector.LastRecordSelector;
-import com.twitter.distributedlog.selector.LogRecordSelector;
-import com.twitter.distributedlog.util.FutureUtils.FutureEventListenerRunnable;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.BoxedUnit;
-
-/**
- * Utility function for readers
- */
-public class ReadUtils {
-
- static final Logger LOG = LoggerFactory.getLogger(ReadUtils.class);
-
- private static final int MIN_SEARCH_BATCH_SIZE = 2;
-
- //
- // Read First & Last Record Functions
- //
-
- /**
- * Read last record from a log segment.
- *
- * @param streamName
- * fully qualified stream name (used for logging)
- * @param l
- * log segment metadata.
- * @param fence
- * whether to fence the log segment.
- * @param includeControl
- * whether to include control record.
- * @param includeEndOfStream
- * whether to include end of stream.
- * @param scanStartBatchSize
- * first num entries used for read last record scan
- * @param scanMaxBatchSize
- * max num entries used for read last record scan
- * @param numRecordsScanned
- * num of records scanned to get last record
- * @param executorService
- * executor service used for processing entries
- * @param entryStore
- * log segment entry store
- * @return a future with last record.
- */
- public static Future<LogRecordWithDLSN> asyncReadLastRecord(
- final String streamName,
- final LogSegmentMetadata l,
- final boolean fence,
- final boolean includeControl,
- final boolean includeEndOfStream,
- final int scanStartBatchSize,
- final int scanMaxBatchSize,
- final AtomicInteger numRecordsScanned,
- final ExecutorService executorService,
- final LogSegmentEntryStore entryStore) {
- final LogRecordSelector selector = new LastRecordSelector();
- return asyncReadRecord(streamName, l, fence, includeControl, includeEndOfStream, scanStartBatchSize,
- scanMaxBatchSize, numRecordsScanned, executorService, entryStore,
- selector, true /* backward */, 0L);
- }
-
- /**
- * Read first record from a log segment with a DLSN larger than that given.
- *
- * @param streamName
- * fully qualified stream name (used for logging)
- * @param l
- * log segment metadata.
- * @param scanStartBatchSize
- * first num entries used for read last record scan
- * @param scanMaxBatchSize
- * max num entries used for read last record scan
- * @param numRecordsScanned
- * num of records scanned to get last record
- * @param executorService
- * executor service used for processing entries
- * @param entryStore
- * log segment entry store
- * @param dlsn
- * threshold dlsn
- * @return a future with last record.
- */
- public static Future<LogRecordWithDLSN> asyncReadFirstUserRecord(
- final String streamName,
- final LogSegmentMetadata l,
- final int scanStartBatchSize,
- final int scanMaxBatchSize,
- final AtomicInteger numRecordsScanned,
- final ExecutorService executorService,
- final LogSegmentEntryStore entryStore,
- final DLSN dlsn) {
- long startEntryId = 0L;
- if (l.getLogSegmentSequenceNumber() == dlsn.getLogSegmentSequenceNo()) {
- startEntryId = dlsn.getEntryId();
- }
- final LogRecordSelector selector = new FirstDLSNNotLessThanSelector(dlsn);
- return asyncReadRecord(streamName, l, false, false, false, scanStartBatchSize,
- scanMaxBatchSize, numRecordsScanned, executorService, entryStore,
- selector, false /* backward */, startEntryId);
- }
-
- //
- // Private methods for scanning log segments
- //
-
- private static class ScanContext {
- // variables to about current scan state
- final AtomicInteger numEntriesToScan;
- final AtomicLong curStartEntryId;
- final AtomicLong curEndEntryId;
-
- // scan settings
- final long startEntryId;
- final long endEntryId;
- final int scanStartBatchSize;
- final int scanMaxBatchSize;
- final boolean includeControl;
- final boolean includeEndOfStream;
- final boolean backward;
-
- // number of records scanned
- final AtomicInteger numRecordsScanned;
-
- ScanContext(long startEntryId, long endEntryId,
- int scanStartBatchSize,
- int scanMaxBatchSize,
- boolean includeControl,
- boolean includeEndOfStream,
- boolean backward,
- AtomicInteger numRecordsScanned) {
- this.startEntryId = startEntryId;
- this.endEntryId = endEntryId;
- this.scanStartBatchSize = scanStartBatchSize;
- this.scanMaxBatchSize = scanMaxBatchSize;
- this.includeControl = includeControl;
- this.includeEndOfStream = includeEndOfStream;
- this.backward = backward;
- // Scan state
- this.numEntriesToScan = new AtomicInteger(scanStartBatchSize);
- if (backward) {
- this.curStartEntryId = new AtomicLong(
- Math.max(startEntryId, (endEntryId - scanStartBatchSize + 1)));
- this.curEndEntryId = new AtomicLong(endEntryId);
- } else {
- this.curStartEntryId = new AtomicLong(startEntryId);
- this.curEndEntryId = new AtomicLong(
- Math.min(endEntryId, (startEntryId + scanStartBatchSize - 1)));
- }
- this.numRecordsScanned = numRecordsScanned;
- }
-
- boolean moveToNextRange() {
- if (backward) {
- return moveBackward();
- } else {
- return moveForward();
- }
- }
-
- boolean moveBackward() {
- long nextEndEntryId = curStartEntryId.get() - 1;
- if (nextEndEntryId < startEntryId) {
- // no entries to read again
- return false;
- }
- curEndEntryId.set(nextEndEntryId);
- // update num entries to scan
- numEntriesToScan.set(
- Math.min(numEntriesToScan.get() * 2, scanMaxBatchSize));
- // update start entry id
- curStartEntryId.set(Math.max(startEntryId, nextEndEntryId - numEntriesToScan.get() + 1));
- return true;
- }
-
- boolean moveForward() {
- long nextStartEntryId = curEndEntryId.get() + 1;
- if (nextStartEntryId > endEntryId) {
- // no entries to read again
- return false;
- }
- curStartEntryId.set(nextStartEntryId);
- // update num entries to scan
- numEntriesToScan.set(
- Math.min(numEntriesToScan.get() * 2, scanMaxBatchSize));
- // update start entry id
- curEndEntryId.set(Math.min(endEntryId, nextStartEntryId + numEntriesToScan.get() - 1));
- return true;
- }
- }
-
- private static class SingleEntryScanContext extends ScanContext {
- SingleEntryScanContext(long entryId) {
- super(entryId, entryId, 1, 1, true, true, false, new AtomicInteger(0));
- }
- }
-
- /**
- * Read record from a given range of log segment entries.
- *
- * @param streamName
- * fully qualified stream name (used for logging)
- * @param reader
- * log segment random access reader
- * @param executorService
- * executor service used for processing entries
- * @param context
- * scan context
- * @return a future with the log record.
- */
- private static Future<LogRecordWithDLSN> asyncReadRecordFromEntries(
- final String streamName,
- final LogSegmentRandomAccessEntryReader reader,
- final LogSegmentMetadata metadata,
- final ExecutorService executorService,
- final ScanContext context,
- final LogRecordSelector selector) {
- final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
- final long startEntryId = context.curStartEntryId.get();
- final long endEntryId = context.curEndEntryId.get();
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} reading entries [{} - {}] from {}.",
- new Object[] { streamName, startEntryId, endEntryId, metadata});
- }
- FutureEventListener<List<Entry.Reader>> readEntriesListener =
- new FutureEventListener<List<Entry.Reader>>() {
- @Override
- public void onSuccess(final List<Entry.Reader> entries) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} finished reading entries [{} - {}] from {}",
- new Object[]{ streamName, startEntryId, endEntryId, metadata});
- }
- for (Entry.Reader entry : entries) {
- try {
- visitEntryRecords(entry, context, selector);
- } catch (IOException ioe) {
- // exception is only thrown due to bad ledger entry, so it might be corrupted
- // we shouldn't do anything beyond this point. throw the exception to application
- promise.setException(ioe);
- return;
- }
- }
-
- LogRecordWithDLSN record = selector.result();
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} got record from entries [{} - {}] of {} : {}",
- new Object[]{streamName, startEntryId, endEntryId,
- metadata, record});
- }
- promise.setValue(record);
- }
-
- @Override
- public void onFailure(final Throwable cause) {
- promise.setException(cause);
- }
- };
- reader.readEntries(startEntryId, endEntryId)
- .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
- return promise;
- }
-
- /**
- * Process each record using LogRecordSelector.
- *
- * @param entry
- * ledger entry
- * @param context
- * scan context
- * @return log record with dlsn inside the ledger entry
- * @throws IOException
- */
- private static void visitEntryRecords(
- Entry.Reader entry,
- ScanContext context,
- LogRecordSelector selector) throws IOException {
- LogRecordWithDLSN nextRecord = entry.nextRecord();
- while (nextRecord != null) {
- LogRecordWithDLSN record = nextRecord;
- nextRecord = entry.nextRecord();
- context.numRecordsScanned.incrementAndGet();
- if (!context.includeControl && record.isControl()) {
- continue;
- }
- if (!context.includeEndOfStream && record.isEndOfStream()) {
- continue;
- }
- selector.process(record);
- }
- }
-
- /**
- * Scan entries for the given record.
- *
- * @param streamName
- * fully qualified stream name (used for logging)
- * @param reader
- * log segment random access reader
- * @param executorService
- * executor service used for processing entries
- * @param promise
- * promise to return desired record.
- * @param context
- * scan context
- */
- private static void asyncReadRecordFromEntries(
- final String streamName,
- final LogSegmentRandomAccessEntryReader reader,
- final LogSegmentMetadata metadata,
- final ExecutorService executorService,
- final Promise<LogRecordWithDLSN> promise,
- final ScanContext context,
- final LogRecordSelector selector) {
- FutureEventListener<LogRecordWithDLSN> readEntriesListener =
- new FutureEventListener<LogRecordWithDLSN>() {
- @Override
- public void onSuccess(LogRecordWithDLSN value) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} read record from [{} - {}] of {} : {}",
- new Object[]{streamName, context.curStartEntryId.get(), context.curEndEntryId.get(),
- metadata, value});
- }
- if (null != value) {
- promise.setValue(value);
- return;
- }
- if (!context.moveToNextRange()) {
- // no entries to read again
- promise.setValue(null);
- return;
- }
- // scan next range
- asyncReadRecordFromEntries(streamName,
- reader,
- metadata,
- executorService,
- promise,
- context,
- selector);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- promise.setException(cause);
- }
- };
- asyncReadRecordFromEntries(streamName, reader, metadata, executorService, context, selector)
- .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
- }
-
- private static void asyncReadRecordFromLogSegment(
- final String streamName,
- final LogSegmentRandomAccessEntryReader reader,
- final LogSegmentMetadata metadata,
- final ExecutorService executorService,
- final int scanStartBatchSize,
- final int scanMaxBatchSize,
- final boolean includeControl,
- final boolean includeEndOfStream,
- final Promise<LogRecordWithDLSN> promise,
- final AtomicInteger numRecordsScanned,
- final LogRecordSelector selector,
- final boolean backward,
- final long startEntryId) {
- final long lastAddConfirmed = reader.getLastAddConfirmed();
- if (lastAddConfirmed < 0) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Log segment {} is empty for {}.", new Object[] { metadata, streamName });
- }
- promise.setValue(null);
- return;
- }
- final ScanContext context = new ScanContext(
- startEntryId, lastAddConfirmed,
- scanStartBatchSize, scanMaxBatchSize,
- includeControl, includeEndOfStream, backward, numRecordsScanned);
- asyncReadRecordFromEntries(streamName, reader, metadata, executorService,
- promise, context, selector);
- }
-
- private static Future<LogRecordWithDLSN> asyncReadRecord(
- final String streamName,
- final LogSegmentMetadata l,
- final boolean fence,
- final boolean includeControl,
- final boolean includeEndOfStream,
- final int scanStartBatchSize,
- final int scanMaxBatchSize,
- final AtomicInteger numRecordsScanned,
- final ExecutorService executorService,
- final LogSegmentEntryStore entryStore,
- final LogRecordSelector selector,
- final boolean backward,
- final long startEntryId) {
-
- final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
-
- FutureEventListener<LogSegmentRandomAccessEntryReader> openReaderListener =
- new FutureEventListener<LogSegmentRandomAccessEntryReader>() {
- @Override
- public void onSuccess(final LogSegmentRandomAccessEntryReader reader) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} Opened log segment {} for reading record",
- streamName, l);
- }
- promise.ensure(new AbstractFunction0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- reader.asyncClose();
- return BoxedUnit.UNIT;
- }
- });
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} {} scanning {}.", new Object[]{
- (backward ? "backward" : "forward"), streamName, l});
- }
- asyncReadRecordFromLogSegment(
- streamName, reader, l, executorService,
- scanStartBatchSize, scanMaxBatchSize,
- includeControl, includeEndOfStream,
- promise, numRecordsScanned, selector, backward, startEntryId);
- }
-
- @Override
- public void onFailure(final Throwable cause) {
- promise.setException(cause);
- }
- };
- entryStore.openRandomAccessReader(l, fence)
- .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
- return promise;
- }
-
- //
- // Search Functions
- //
-
- /**
- * Get the log record whose transaction id is not less than provided <code>transactionId</code>.
- *
- * <p>
- * It uses a binary-search like algorithm to find the log record whose transaction id is not less than
- * provided <code>transactionId</code> within a log <code>segment</code>. You could think of a log segment
- * in terms of a sequence of records whose transaction ids are non-decreasing.
- *
- * - The sequence of records within a log segment is divided into N pieces.
- * - Find the piece of records that contains a record whose transaction id is not less than provided
- * <code>transactionId</code>.
- *
- * N could be chosen based on trading off concurrency and latency.
- * </p>
- *
- * @param logName
- * name of the log
- * @param segment
- * metadata of the log segment
- * @param transactionId
- * transaction id
- * @param executorService
- * executor service used for processing entries
- * @param entryStore
- * log segment entry store
- * @param nWays
- * how many number of entries to search in parallel
- * @return found log record. none if all transaction ids are less than provided <code>transactionId</code>.
- */
- public static Future<Optional<LogRecordWithDLSN>> getLogRecordNotLessThanTxId(
- final String logName,
- final LogSegmentMetadata segment,
- final long transactionId,
- final ExecutorService executorService,
- final LogSegmentEntryStore entryStore,
- final int nWays) {
- if (!segment.isInProgress()) {
- if (segment.getLastTxId() < transactionId) {
- // all log records whose transaction id is less than provided transactionId
- // then return none
- Optional<LogRecordWithDLSN> noneRecord = Optional.absent();
- return Future.value(noneRecord);
- }
- }
-
- final Promise<Optional<LogRecordWithDLSN>> promise =
- new Promise<Optional<LogRecordWithDLSN>>();
- final FutureEventListener<LogSegmentRandomAccessEntryReader> openReaderListener =
- new FutureEventListener<LogSegmentRandomAccessEntryReader>() {
- @Override
- public void onSuccess(final LogSegmentRandomAccessEntryReader reader) {
- promise.ensure(new AbstractFunction0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- reader.asyncClose();
- return BoxedUnit.UNIT;
- }
-
- });
- long lastEntryId = reader.getLastAddConfirmed();
- if (lastEntryId < 0) {
- // it means that the log segment is created but not written yet or an empty log segment.
- // it is equivalent to 'all log records whose transaction id is less than provided transactionId'
- Optional<LogRecordWithDLSN> nonRecord = Optional.absent();
- promise.setValue(nonRecord);
- return;
- }
- // all log records whose transaction id is not less than provided transactionId
- if (segment.getFirstTxId() >= transactionId) {
- final FirstTxIdNotLessThanSelector selector =
- new FirstTxIdNotLessThanSelector(transactionId);
- asyncReadRecordFromEntries(
- logName,
- reader,
- segment,
- executorService,
- new SingleEntryScanContext(0L),
- selector
- ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
- @Override
- public void onSuccess(LogRecordWithDLSN value) {
- promise.setValue(Optional.of(selector.result()));
- }
-
- @Override
- public void onFailure(Throwable cause) {
- promise.setException(cause);
- }
- });
-
- return;
- }
- getLogRecordNotLessThanTxIdFromEntries(
- logName,
- segment,
- transactionId,
- executorService,
- reader,
- Lists.newArrayList(0L, lastEntryId),
- nWays,
- Optional.<LogRecordWithDLSN>absent(),
- promise);
- }
-
- @Override
- public void onFailure(final Throwable cause) {
- promise.setException(cause);
- }
- };
-
- entryStore.openRandomAccessReader(segment, false)
- .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
- return promise;
- }
-
- /**
- * Find the log record whose transaction id is not less than provided <code>transactionId</code> from
- * entries between <code>startEntryId</code> and <code>endEntryId</code>.
- *
- * @param logName
- * name of the log
- * @param segment
- * log segment
- * @param transactionId
- * provided transaction id to search
- * @param executorService
- * executor service
- * @param reader
- * log segment random access reader
- * @param entriesToSearch
- * list of entries to search
- * @param nWays
- * how many entries to search in parallel
- * @param prevFoundRecord
- * the log record found in previous search
- * @param promise
- * promise to satisfy the result
- */
- private static void getLogRecordNotLessThanTxIdFromEntries(
- final String logName,
- final LogSegmentMetadata segment,
- final long transactionId,
- final ExecutorService executorService,
- final LogSegmentRandomAccessEntryReader reader,
- final List<Long> entriesToSearch,
- final int nWays,
- final Optional<LogRecordWithDLSN> prevFoundRecord,
- final Promise<Optional<LogRecordWithDLSN>> promise) {
- final List<Future<LogRecordWithDLSN>> searchResults =
- Lists.newArrayListWithExpectedSize(entriesToSearch.size());
- for (Long entryId : entriesToSearch) {
- LogRecordSelector selector = new FirstTxIdNotLessThanSelector(transactionId);
- Future<LogRecordWithDLSN> searchResult = asyncReadRecordFromEntries(
- logName,
- reader,
- segment,
- executorService,
- new SingleEntryScanContext(entryId),
- selector);
- searchResults.add(searchResult);
- }
- FutureEventListener<List<LogRecordWithDLSN>> processSearchResultsListener =
- new FutureEventListener<List<LogRecordWithDLSN>>() {
- @Override
- public void onSuccess(List<LogRecordWithDLSN> resultList) {
- processSearchResults(
- logName,
- segment,
- transactionId,
- executorService,
- reader,
- resultList,
- nWays,
- prevFoundRecord,
- promise);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- promise.setException(cause);
- }
- };
- Future.collect(searchResults).addEventListener(
- FutureEventListenerRunnable.of(processSearchResultsListener, executorService));
- }
-
- /**
- * Process the search results
- */
- static void processSearchResults(
- final String logName,
- final LogSegmentMetadata segment,
- final long transactionId,
- final ExecutorService executorService,
- final LogSegmentRandomAccessEntryReader reader,
- final List<LogRecordWithDLSN> searchResults,
- final int nWays,
- final Optional<LogRecordWithDLSN> prevFoundRecord,
- final Promise<Optional<LogRecordWithDLSN>> promise) {
- int found = -1;
- for (int i = 0; i < searchResults.size(); i++) {
- LogRecordWithDLSN record = searchResults.get(i);
- if (record.getTransactionId() >= transactionId) {
- found = i;
- break;
- }
- }
- if (found == -1) { // all log records' transaction id is less than provided transaction id
- promise.setValue(prevFoundRecord);
- return;
- }
- // we found a log record
- LogRecordWithDLSN foundRecord = searchResults.get(found);
-
- // we found it
- // - it is not the first record
- // - it is the first record in first search entry
- // - its entry is adjacent to previous search entry
- if (foundRecord.getDlsn().getSlotId() != 0L
- || found == 0
- || foundRecord.getDlsn().getEntryId() == (searchResults.get(found - 1).getDlsn().getEntryId() + 1)) {
- promise.setValue(Optional.of(foundRecord));
- return;
- }
-
- // otherwise, we need to search
- List<Long> nextSearchBatch = getEntriesToSearch(
- transactionId,
- searchResults.get(found - 1),
- searchResults.get(found),
- nWays);
- if (nextSearchBatch.isEmpty()) {
- promise.setValue(prevFoundRecord);
- return;
- }
- getLogRecordNotLessThanTxIdFromEntries(
- logName,
- segment,
- transactionId,
- executorService,
- reader,
- nextSearchBatch,
- nWays,
- Optional.of(foundRecord),
- promise);
- }
-
- /**
- * Get the entries to search provided <code>transactionId</code> between
- * <code>firstRecord</code> and <code>lastRecord</code>. <code>firstRecord</code>
- * and <code>lastRecord</code> are already searched, which the transaction id
- * of <code>firstRecord</code> is less than <code>transactionId</code> and the
- * transaction id of <code>lastRecord</code> is not less than <code>transactionId</code>.
- *
- * @param transactionId
- * transaction id to search
- * @param firstRecord
- * log record that already searched whose transaction id is leass than <code>transactionId</code>.
- * @param lastRecord
- * log record that already searched whose transaction id is not less than <code>transactionId</code>.
- * @param nWays
- * N-ways to search
- * @return the list of entries to search
- */
- static List<Long> getEntriesToSearch(
- long transactionId,
- LogRecordWithDLSN firstRecord,
- LogRecordWithDLSN lastRecord,
- int nWays) {
- long txnDiff = lastRecord.getTransactionId() - firstRecord.getTransactionId();
- if (txnDiff > 0) {
- if (lastRecord.getTransactionId() == transactionId) {
- List<Long> entries = getEntriesToSearch(
- firstRecord.getDlsn().getEntryId() + 1,
- lastRecord.getDlsn().getEntryId() - 2,
- Math.max(MIN_SEARCH_BATCH_SIZE, nWays - 1));
- entries.add(lastRecord.getDlsn().getEntryId() - 1);
- return entries;
- } else {
- // TODO: improve it by estimating transaction ids.
- return getEntriesToSearch(
- firstRecord.getDlsn().getEntryId() + 1,
- lastRecord.getDlsn().getEntryId() - 1,
- nWays);
- }
- } else {
- // unexpected condition
- return Lists.newArrayList();
- }
- }
-
- static List<Long> getEntriesToSearch(
- long startEntryId,
- long endEntryId,
- int nWays) {
- if (startEntryId > endEntryId) {
- return Lists.newArrayList();
- }
- long numEntries = endEntryId - startEntryId + 1;
- long step = Math.max(1L, numEntries / nWays);
- List<Long> entryList = Lists.newArrayListWithExpectedSize(nWays);
- for (long i = startEntryId, j = nWays - 1; i <= endEntryId && j > 0; i += step, j--) {
- entryList.add(i);
- }
- if (entryList.get(entryList.size() - 1) < endEntryId) {
- entryList.add(endEntryId);
- }
- return entryList;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/WriteLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/WriteLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/WriteLimiter.java
deleted file mode 100644
index 0b24c1a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/WriteLimiter.java
+++ /dev/null
@@ -1,62 +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.OverCapacityException;
-import com.twitter.distributedlog.util.PermitLimiter;
-
-public class WriteLimiter {
-
- String streamName;
- final PermitLimiter streamLimiter;
- final PermitLimiter globalLimiter;
-
- public WriteLimiter(String streamName, PermitLimiter streamLimiter, PermitLimiter globalLimiter) {
- this.streamName = streamName;
- this.streamLimiter = streamLimiter;
- this.globalLimiter = globalLimiter;
- }
-
- public void acquire() throws OverCapacityException {
- if (!streamLimiter.acquire()) {
- throw new OverCapacityException(String.format("Stream write capacity exceeded for stream %s", streamName));
- }
- try {
- if (!globalLimiter.acquire()) {
- throw new OverCapacityException("Global write capacity exceeded");
- }
- } catch (OverCapacityException ex) {
- streamLimiter.release(1);
- throw ex;
- }
- }
-
- public void release() {
- release(1);
- }
-
- public void release(int permits) {
- streamLimiter.release(permits);
- globalLimiter.release(permits);
- }
-
- public void close() {
- streamLimiter.close();
- globalLimiter.close();
- }
-}
[29/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LogSegmentMetadata.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogSegmentMetadata.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LogSegmentMetadata.java
deleted file mode 100644
index a8d9e6d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogSegmentMetadata.java
+++ /dev/null
@@ -1,1125 +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 java.io.File;
-import java.io.IOException;
-import java.util.Comparator;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.UnsupportedMetadataVersionException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Utility class for storing the metadata associated
- * with a single edit log segment, stored in a single ledger
- */
-public class LogSegmentMetadata {
- static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadata.class);
-
- public static enum LogSegmentMetadataVersion {
- VERSION_INVALID(0),
- VERSION_V1_ORIGINAL(1),
- VERSION_V2_LEDGER_SEQNO(2),
- VERSION_V3_MIN_ACTIVE_DLSN(3),
- VERSION_V4_ENVELOPED_ENTRIES(4),
- VERSION_V5_SEQUENCE_ID(5);
-
- public final int value;
-
- private LogSegmentMetadataVersion(int value) {
- this.value = value;
- }
-
- public static LogSegmentMetadataVersion of(int version) {
- switch (version) {
- case 5:
- return VERSION_V5_SEQUENCE_ID;
- case 4:
- return VERSION_V4_ENVELOPED_ENTRIES;
- case 3:
- return VERSION_V3_MIN_ACTIVE_DLSN;
- case 2:
- return VERSION_V2_LEDGER_SEQNO;
- case 1:
- return VERSION_V1_ORIGINAL;
- case 0:
- return VERSION_INVALID;
- default:
- throw new IllegalArgumentException("unknown version " + version);
- }
- }
- }
-
- public static enum TruncationStatus {
- ACTIVE (0), PARTIALLY_TRUNCATED(1), TRUNCATED (2);
- private final int value;
-
- private TruncationStatus(int value) {
- this.value = value;
- }
- }
-
- public static class LogSegmentMetadataBuilder {
- protected String zkPath;
- protected long logSegmentId;
- protected LogSegmentMetadataVersion version;
- protected long firstTxId;
- protected int regionId;
- protected long status;
- protected long lastTxId;
- protected long completionTime;
- protected int recordCount;
- protected long logSegmentSequenceNo;
- protected long lastEntryId;
- protected long lastSlotId;
- protected long minActiveEntryId;
- protected long minActiveSlotId;
- protected long startSequenceId;
- protected boolean inprogress;
-
- // This is a derived attribute.
- // Since we overwrite the original version with the target version, information that is
- // derived from the original version (e.g. does it support enveloping of entries)
- // is lost while parsing.
- // NOTE: This value is not stored in the Metadata store.
- protected boolean envelopeEntries = false;
-
- LogSegmentMetadataBuilder(String zkPath,
- LogSegmentMetadataVersion version,
- long logSegmentId,
- long firstTxId) {
- initialize();
- this.zkPath = zkPath;
- this.version = version;
- this.logSegmentId = logSegmentId;
- this.firstTxId = firstTxId;
- }
-
- LogSegmentMetadataBuilder(String zkPath,
- int version,
- long logSegmentId,
- long firstTxId) {
- this(zkPath, LogSegmentMetadataVersion.values()[version], logSegmentId, firstTxId);
- }
-
- private void initialize() {
- regionId = DistributedLogConstants.LOCAL_REGION_ID;
- status = DistributedLogConstants.LOGSEGMENT_DEFAULT_STATUS;
- lastTxId = DistributedLogConstants.INVALID_TXID;
- completionTime = 0;
- recordCount = 0;
- lastEntryId = -1;
- lastSlotId = -1;
- minActiveEntryId = 0;
- minActiveSlotId = 0;
- startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
- inprogress = true;
- }
-
- LogSegmentMetadataBuilder setRegionId(int regionId) {
- this.regionId = regionId;
- return this;
- }
-
- LogSegmentMetadataBuilder setStatus(long status) {
- this.status = status;
- return this;
- }
-
- public LogSegmentMetadataBuilder setLastTxId(long lastTxId) {
- this.lastTxId = lastTxId;
- return this;
- }
-
- public LogSegmentMetadataBuilder setCompletionTime(long completionTime) {
- this.completionTime = completionTime;
- return this;
- }
-
- public LogSegmentMetadataBuilder setRecordCount(int recordCount) {
- this.recordCount = recordCount;
- return this;
- }
-
- public LogSegmentMetadataBuilder setRecordCount(LogRecord record) {
- this.recordCount = record.getLastPositionWithinLogSegment();
- return this;
- }
-
- public LogSegmentMetadataBuilder setInprogress(boolean inprogress) {
- this.inprogress = inprogress;
- return this;
- }
-
- LogSegmentMetadataBuilder setLogSegmentSequenceNo(long logSegmentSequenceNo) {
- this.logSegmentSequenceNo = logSegmentSequenceNo;
- return this;
- }
-
- public LogSegmentMetadataBuilder setLastEntryId(long lastEntryId) {
- this.lastEntryId = lastEntryId;
- return this;
- }
-
- LogSegmentMetadataBuilder setLastSlotId(long lastSlotId) {
- this.lastSlotId = lastSlotId;
- return this;
- }
-
- LogSegmentMetadataBuilder setEnvelopeEntries(boolean envelopeEntries) {
- this.envelopeEntries = envelopeEntries;
- return this;
- }
-
- LogSegmentMetadataBuilder setMinActiveEntryId(long minActiveEntryId) {
- this.minActiveEntryId = minActiveEntryId;
- return this;
- }
-
- LogSegmentMetadataBuilder setMinActiveSlotId(long minActiveSlotId) {
- this.minActiveSlotId = minActiveSlotId;
- return this;
- }
-
- LogSegmentMetadataBuilder setStartSequenceId(long startSequenceId) {
- this.startSequenceId = startSequenceId;
- return this;
- }
-
- public LogSegmentMetadata build() {
- return new LogSegmentMetadata(
- zkPath,
- version,
- logSegmentId,
- firstTxId,
- lastTxId,
- completionTime,
- inprogress,
- recordCount,
- logSegmentSequenceNo,
- lastEntryId,
- lastSlotId,
- regionId,
- status,
- minActiveEntryId,
- minActiveSlotId,
- startSequenceId,
- envelopeEntries
- );
- }
-
- }
-
- /**
- * Mutator to mutate the metadata of a log segment. This mutator is going to create
- * a new instance of the log segment metadata without changing the existing one.
- */
- public static class Mutator extends LogSegmentMetadataBuilder {
-
- Mutator(LogSegmentMetadata original) {
- super(original.getZkPath(), original.getVersion(), original.getLogSegmentId(), original.getFirstTxId());
- this.inprogress = original.isInProgress();
- this.logSegmentSequenceNo = original.getLogSegmentSequenceNumber();
- this.lastEntryId = original.getLastEntryId();
- this.lastSlotId = original.getLastSlotId();
- this.lastTxId = original.getLastTxId();
- this.completionTime = original.getCompletionTime();
- this.recordCount = original.getRecordCount();
- this.regionId = original.getRegionId();
- this.status = original.getStatus();
- this.minActiveEntryId = original.getMinActiveDLSN().getEntryId();
- this.minActiveSlotId = original.getMinActiveDLSN().getSlotId();
- this.startSequenceId = original.getStartSequenceId();
- this.envelopeEntries = original.getEnvelopeEntries();
- }
-
- @VisibleForTesting
- public Mutator setVersion(LogSegmentMetadataVersion version) {
- this.version = version;
- return this;
- }
-
- public Mutator setLogSegmentSequenceNumber(long seqNo) {
- this.logSegmentSequenceNo = seqNo;
- return this;
- }
-
- public Mutator setZkPath(String zkPath) {
- this.zkPath = zkPath;
- return this;
- }
-
- public Mutator setLastDLSN(DLSN dlsn) {
- this.logSegmentSequenceNo = dlsn.getLogSegmentSequenceNo();
- this.lastEntryId = dlsn.getEntryId();
- this.lastSlotId = dlsn.getSlotId();
- return this;
- }
-
- public Mutator setMinActiveDLSN(DLSN dlsn) {
- if (this.logSegmentSequenceNo != dlsn.getLogSegmentSequenceNo()) {
- throw new IllegalArgumentException("Updating minDLSN in an incorrect log segment");
- }
- this.minActiveEntryId = dlsn.getEntryId();
- this.minActiveSlotId = dlsn.getSlotId();
- return this;
- }
-
- public Mutator setTruncationStatus(TruncationStatus truncationStatus) {
- status &= ~METADATA_TRUNCATION_STATUS_MASK;
- status |= (truncationStatus.value & METADATA_TRUNCATION_STATUS_MASK);
- return this;
- }
-
- public Mutator setStartSequenceId(long startSequenceId) {
- this.startSequenceId = startSequenceId;
- return this;
- }
- }
-
- private final String zkPath;
- private final long logSegmentId;
- private final LogSegmentMetadataVersion version;
- private final long firstTxId;
- private final int regionId;
- private final long status;
- private final long lastTxId;
- private final long completionTime;
- private final int recordCount;
- private final DLSN lastDLSN;
- private final DLSN minActiveDLSN;
- private final long startSequenceId;
- private final boolean inprogress;
- // This is a derived attribute.
- // Since we overwrite the original version with the target version, information that is
- // derived from the original version (e.g. does it support enveloping of entries)
- // is lost while parsing.
- // NOTE: This value is not stored in the Metadata store.
- private final boolean envelopeEntries;
-
- public static final Comparator<LogSegmentMetadata> COMPARATOR
- = new Comparator<LogSegmentMetadata>() {
-
- public int compare(LogSegmentMetadata o1,
- LogSegmentMetadata o2) {
- if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
- (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
- if (o1.firstTxId < o2.firstTxId) {
- return -1;
- } else if (o1.firstTxId == o2.firstTxId) {
- return 0;
- } else {
- return 1;
- }
- } else {
- if (o1.getLogSegmentSequenceNumber() < o2.getLogSegmentSequenceNumber()) {
- return -1;
- } else if (o1.getLogSegmentSequenceNumber() == o2.getLogSegmentSequenceNumber()) {
- // make sure we won't move over inprogress log segment if it still presents in the list
- if (o1.isInProgress() && !o2.isInProgress()) {
- return -1;
- } else if (!o1.isInProgress() && o2.isInProgress()) {
- return 1;
- } else {
- return 0;
- }
- } else {
- return 1;
- }
- }
-
-
- }
- };
-
- public static final Comparator<LogSegmentMetadata> DESC_COMPARATOR
- = new Comparator<LogSegmentMetadata>() {
- public int compare(LogSegmentMetadata o1,
- LogSegmentMetadata o2) {
- if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
- (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
- if (o1.firstTxId > o2.firstTxId) {
- return -1;
- } else if (o1.firstTxId == o2.firstTxId) {
- return 0;
- } else {
- return 1;
- }
- } else {
- if (o1.getLogSegmentSequenceNumber() > o2.getLogSegmentSequenceNumber()) {
- return -1;
- } else if (o1.getLogSegmentSequenceNumber() == o2.getLogSegmentSequenceNumber()) {
- // make sure we won't move over inprogress log segment if it still presents in the list
- if (o1.isInProgress() && !o2.isInProgress()) {
- return 1;
- } else if (!o1.isInProgress() && o2.isInProgress()) {
- return -1;
- } else {
- return 0;
- }
- } else {
- return 1;
- }
- }
- }
- };
-
- public static final int LEDGER_METADATA_CURRENT_LAYOUT_VERSION =
- LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
-
- public static final int LEDGER_METADATA_OLDEST_SUPPORTED_VERSION =
- LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value;
-
- static final int LOGRECORD_COUNT_SHIFT = 32;
- static final long LOGRECORD_COUNT_MASK = 0xffffffff00000000L;
- static final int REGION_SHIFT = 28;
- static final long MAX_REGION_ID = 0xfL;
- static final long REGION_MASK = 0x00000000f0000000L;
- static final int STATUS_BITS_SHIFT = 8;
- static final long STATUS_BITS_MASK = 0x000000000000ff00L;
- static final long UNUSED_BITS_MASK = 0x000000000fff0000L;
- static final long METADATA_VERSION_MASK = 0x00000000000000ffL;
-
- //Metadata status bits
- static final long METADATA_TRUNCATION_STATUS_MASK = 0x3L;
- static final long METADATA_STATUS_BIT_MAX = 0xffL;
-
- private LogSegmentMetadata(String zkPath,
- LogSegmentMetadataVersion version,
- long logSegmentId,
- long firstTxId,
- long lastTxId,
- long completionTime,
- boolean inprogress,
- int recordCount,
- long logSegmentSequenceNumber,
- long lastEntryId,
- long lastSlotId,
- int regionId,
- long status,
- long minActiveEntryId,
- long minActiveSlotId,
- long startSequenceId,
- boolean envelopeEntries) {
- this.zkPath = zkPath;
- this.logSegmentId = logSegmentId;
- this.version = version;
- this.firstTxId = firstTxId;
- this.lastTxId = lastTxId;
- this.inprogress = inprogress;
- this.completionTime = completionTime;
- this.recordCount = recordCount;
- this.lastDLSN = new DLSN(logSegmentSequenceNumber, lastEntryId, lastSlotId);
- this.minActiveDLSN = new DLSN(logSegmentSequenceNumber, minActiveEntryId, minActiveSlotId);
- this.startSequenceId = startSequenceId;
- this.regionId = regionId;
- this.status = status;
- this.envelopeEntries = envelopeEntries;
- }
-
- public String getZkPath() {
- return zkPath;
- }
-
- public String getZNodeName() {
- return new File(zkPath).getName();
- }
-
- public long getFirstTxId() {
- return firstTxId;
- }
-
- public long getLastTxId() {
- return lastTxId;
- }
-
- public long getCompletionTime() {
- return completionTime;
- }
-
- public long getLogSegmentId() {
- return logSegmentId;
- }
-
- public long getLogSegmentSequenceNumber() {
- return lastDLSN.getLogSegmentSequenceNo();
- }
-
- public int getVersion() {
- return version.value;
- }
-
- public boolean getEnvelopeEntries() {
- return envelopeEntries;
- }
-
- public long getLastEntryId() {
- return lastDLSN.getEntryId();
- }
-
- long getStatus() {
- return status;
- }
-
- public long getStartSequenceId() {
- // generate negative sequence id for log segments that created <= v4
- return supportsSequenceId() && startSequenceId != DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ?
- startSequenceId : Long.MIN_VALUE + (getLogSegmentSequenceNumber() << 32L);
- }
-
- public boolean isTruncated() {
- return ((status & METADATA_TRUNCATION_STATUS_MASK)
- == TruncationStatus.TRUNCATED.value);
- }
-
- public boolean isPartiallyTruncated() {
- return ((status & METADATA_TRUNCATION_STATUS_MASK)
- == TruncationStatus.PARTIALLY_TRUNCATED.value);
- }
-
- public boolean isNonTruncated() {
- return ((status & METADATA_TRUNCATION_STATUS_MASK)
- == TruncationStatus.ACTIVE.value);
- }
-
- public long getLastSlotId() {
- return lastDLSN.getSlotId();
- }
-
- public DLSN getLastDLSN() {
- return lastDLSN;
- }
-
- public DLSN getMinActiveDLSN() {
- return minActiveDLSN;
- }
-
- public DLSN getFirstDLSN() {
- return new DLSN(getLogSegmentSequenceNumber(), 0, 0);
- }
-
- public int getRecordCount() {
- return recordCount;
- }
-
- public int getRegionId() {
- return regionId;
- }
-
- public boolean isInProgress() {
- return this.inprogress;
- }
-
- @VisibleForTesting
- public boolean isDLSNinThisSegment(DLSN dlsn) {
- return dlsn.getLogSegmentSequenceNo() == getLogSegmentSequenceNumber();
- }
-
- @VisibleForTesting
- public boolean isRecordPositionWithinSegmentScope(LogRecord record) {
- return record.getLastPositionWithinLogSegment() <= getRecordCount();
- }
-
- @VisibleForTesting
- public boolean isRecordLastPositioninThisSegment(LogRecord record) {
- return record.getLastPositionWithinLogSegment() == getRecordCount();
- }
-
- /**
- * complete current log segment. A new log segment metadata instance will be returned.
- *
- * @param zkPath
- * zk path for the completed log segment.
- * @param newLastTxId
- * last tx id
- * @param recordCount
- * record count
- * @param lastEntryId
- * last entry id
- * @param lastSlotId
- * last slot id
- * @return completed log segment.
- */
- LogSegmentMetadata completeLogSegment(String zkPath,
- long newLastTxId,
- int recordCount,
- long lastEntryId,
- long lastSlotId,
- long startSequenceId) {
- assert this.lastTxId == DistributedLogConstants.INVALID_TXID;
-
- return new Mutator(this)
- .setZkPath(zkPath)
- .setLastDLSN(new DLSN(this.lastDLSN.getLogSegmentSequenceNo(), lastEntryId, lastSlotId))
- .setLastTxId(newLastTxId)
- .setInprogress(false)
- .setCompletionTime(Utils.nowInMillis())
- .setRecordCount(recordCount)
- .setStartSequenceId(startSequenceId)
- .build();
- }
-
- public static Future<LogSegmentMetadata> read(ZooKeeperClient zkc, String path) {
- return read(zkc, path, false);
- }
-
- public static Future<LogSegmentMetadata> read(ZooKeeperClient zkc, String path, final boolean skipMinVersionCheck) {
- final Promise<LogSegmentMetadata> result = new Promise<LogSegmentMetadata>();
- try {
- zkc.get().getData(path, false, new AsyncCallback.DataCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
- if (KeeperException.Code.OK.intValue() != rc) {
- if (KeeperException.Code.NONODE.intValue() == rc) {
- FutureUtils.setException(result, new LogSegmentNotFoundException(path));
- } else {
- FutureUtils.setException(result,
- new ZKException("Failed to read log segment metadata from " + path,
- KeeperException.Code.get(rc)));
- }
- return;
- }
- try {
- LogSegmentMetadata metadata = parseData(path, data, skipMinVersionCheck);
- FutureUtils.setValue(result, metadata);
- } catch (IOException ie) {
- LOG.error("Error on parsing log segment metadata from {} : ", path, ie);
- result.setException(ie);
- }
- }
- }, null);
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- result.setException(FutureUtils.zkException(e, path));
- } catch (InterruptedException e) {
- result.setException(FutureUtils.zkException(e, path));
- }
- return result;
- }
-
- static LogSegmentMetadata parseDataV1(String path, byte[] data, String[] parts)
- throws IOException {
- long versionStatusCount = Long.parseLong(parts[0]);
-
- long version = versionStatusCount & METADATA_VERSION_MASK;
- assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
- assert (1 == version);
-
- LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V1_ORIGINAL;
-
- int regionId = (int)(versionStatusCount & REGION_MASK) >> REGION_SHIFT;
- assert (regionId >= 0 && regionId <= 0xf);
-
- long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
- assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
-
- if (parts.length == 3) {
- long logSegmentId = Long.parseLong(parts[1]);
- long txId = Long.parseLong(parts[2]);
- return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
- .setRegionId(regionId)
- .setStatus(status)
- .build();
- } else if (parts.length == 5) {
- long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
- assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
-
- long logSegmentId = Long.parseLong(parts[1]);
- long firstTxId = Long.parseLong(parts[2]);
- long lastTxId = Long.parseLong(parts[3]);
- long completionTime = Long.parseLong(parts[4]);
- return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
- .setInprogress(false)
- .setLastTxId(lastTxId)
- .setCompletionTime(completionTime)
- .setRecordCount((int) recordCount)
- .setRegionId(regionId)
- .setStatus(status)
- .build();
- } else {
- throw new IOException("Invalid log segment metadata : "
- + new String(data, UTF_8));
- }
- }
-
- static LogSegmentMetadata parseDataV2(String path, byte[] data, String[] parts)
- throws IOException {
- long versionStatusCount = Long.parseLong(parts[0]);
-
- long version = versionStatusCount & METADATA_VERSION_MASK;
- assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
- assert (2 == version);
-
- LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO;
-
- int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
- assert (regionId >= 0 && regionId <= 0xf);
-
- long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
- assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
-
- if (parts.length == 4) {
- long logSegmentId = Long.parseLong(parts[1]);
- long txId = Long.parseLong(parts[2]);
- long logSegmentSequenceNumber = Long.parseLong(parts[3]);
- return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
- .setLogSegmentSequenceNo(logSegmentSequenceNumber)
- .setRegionId(regionId)
- .setStatus(status)
- .build();
- } else if (parts.length == 8) {
- long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
- assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
-
- long logSegmentId = Long.parseLong(parts[1]);
- long firstTxId = Long.parseLong(parts[2]);
- long lastTxId = Long.parseLong(parts[3]);
- long completionTime = Long.parseLong(parts[4]);
- long logSegmentSequenceNumber = Long.parseLong(parts[5]);
- long lastEntryId = Long.parseLong(parts[6]);
- long lastSlotId = Long.parseLong(parts[7]);
- return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
- .setInprogress(false)
- .setLastTxId(lastTxId)
- .setCompletionTime(completionTime)
- .setRecordCount((int) recordCount)
- .setLogSegmentSequenceNo(logSegmentSequenceNumber)
- .setLastEntryId(lastEntryId)
- .setLastSlotId(lastSlotId)
- .setRegionId(regionId)
- .setStatus(status)
- .build();
- } else {
- throw new IOException("Invalid logsegment metadata : "
- + new String(data, UTF_8));
- }
-
- }
-
- static LogSegmentMetadata parseDataVersionsWithMinActiveDLSN(String path, byte[] data, String[] parts)
- throws IOException {
- long versionStatusCount = Long.parseLong(parts[0]);
-
- long version = versionStatusCount & METADATA_VERSION_MASK;
- assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
- assert (LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version &&
- LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version);
-
- LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
-
- int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
- assert (regionId >= 0 && regionId <= 0xf);
-
- long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
- assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
-
- if (parts.length == 6) {
- long logSegmentId = Long.parseLong(parts[1]);
- long txId = Long.parseLong(parts[2]);
- long logSegmentSequenceNumber = Long.parseLong(parts[3]);
- long minActiveEntryId = Long.parseLong(parts[4]);
- long minActiveSlotId = Long.parseLong(parts[5]);
-
- LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
- .setLogSegmentSequenceNo(logSegmentSequenceNumber)
- .setMinActiveEntryId(minActiveEntryId)
- .setMinActiveSlotId(minActiveSlotId)
- .setRegionId(regionId)
- .setStatus(status);
- if (supportsEnvelopedEntries((int) version)) {
- builder = builder.setEnvelopeEntries(true);
- }
- return builder.build();
- } else if (parts.length == 10) {
- long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
- assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
-
- long logSegmentId = Long.parseLong(parts[1]);
- long firstTxId = Long.parseLong(parts[2]);
- long lastTxId = Long.parseLong(parts[3]);
- long completionTime = Long.parseLong(parts[4]);
- long logSegmentSequenceNumber = Long.parseLong(parts[5]);
- long lastEntryId = Long.parseLong(parts[6]);
- long lastSlotId = Long.parseLong(parts[7]);
- long minActiveEntryId = Long.parseLong(parts[8]);
- long minActiveSlotId = Long.parseLong(parts[9]);
- LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
- .setInprogress(false)
- .setLastTxId(lastTxId)
- .setCompletionTime(completionTime)
- .setRecordCount((int) recordCount)
- .setLogSegmentSequenceNo(logSegmentSequenceNumber)
- .setLastEntryId(lastEntryId)
- .setLastSlotId(lastSlotId)
- .setMinActiveEntryId(minActiveEntryId)
- .setMinActiveSlotId(minActiveSlotId)
- .setRegionId(regionId)
- .setStatus(status);
- if (supportsEnvelopedEntries((int) version)) {
- builder = builder.setEnvelopeEntries(true);
- }
- return builder.build();
- } else {
- throw new IOException("Invalid logsegment metadata : "
- + new String(data, UTF_8));
- }
-
- }
-
- static LogSegmentMetadata parseDataVersionsWithSequenceId(String path, byte[] data, String[] parts)
- throws IOException {
- long versionStatusCount = Long.parseLong(parts[0]);
-
- long version = versionStatusCount & METADATA_VERSION_MASK;
- assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
- assert (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version &&
- LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version);
-
- LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
-
- int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
- assert (regionId >= 0 && regionId <= 0xf);
-
- long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
- assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
-
- if (parts.length == 7) {
- long logSegmentId = Long.parseLong(parts[1]);
- long txId = Long.parseLong(parts[2]);
- long logSegmentSequenceNumber = Long.parseLong(parts[3]);
- long minActiveEntryId = Long.parseLong(parts[4]);
- long minActiveSlotId = Long.parseLong(parts[5]);
- long startSequenceId = Long.parseLong(parts[6]);
-
- LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
- .setLogSegmentSequenceNo(logSegmentSequenceNumber)
- .setMinActiveEntryId(minActiveEntryId)
- .setMinActiveSlotId(minActiveSlotId)
- .setRegionId(regionId)
- .setStatus(status)
- .setStartSequenceId(startSequenceId)
- .setEnvelopeEntries(true);
- return builder.build();
- } else if (parts.length == 11) {
- long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
- assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
-
- long logSegmentId = Long.parseLong(parts[1]);
- long firstTxId = Long.parseLong(parts[2]);
- long lastTxId = Long.parseLong(parts[3]);
- long completionTime = Long.parseLong(parts[4]);
- long logSegmentSequenceNumber = Long.parseLong(parts[5]);
- long lastEntryId = Long.parseLong(parts[6]);
- long lastSlotId = Long.parseLong(parts[7]);
- long minActiveEntryId = Long.parseLong(parts[8]);
- long minActiveSlotId = Long.parseLong(parts[9]);
- long startSequenceId = Long.parseLong(parts[10]);
- LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
- .setInprogress(false)
- .setLastTxId(lastTxId)
- .setCompletionTime(completionTime)
- .setRecordCount((int) recordCount)
- .setLogSegmentSequenceNo(logSegmentSequenceNumber)
- .setLastEntryId(lastEntryId)
- .setLastSlotId(lastSlotId)
- .setMinActiveEntryId(minActiveEntryId)
- .setMinActiveSlotId(minActiveSlotId)
- .setRegionId(regionId)
- .setStatus(status)
- .setStartSequenceId(startSequenceId)
- .setEnvelopeEntries(true);
- return builder.build();
- } else {
- throw new IOException("Invalid log segment metadata : "
- + new String(data, UTF_8));
- }
- }
-
- public static LogSegmentMetadata parseData(String path, byte[] data)
- throws IOException {
- return parseData(path, data, false);
- }
-
- static LogSegmentMetadata parseData(String path, byte[] data, boolean skipMinVersionCheck) throws IOException {
- String[] parts = new String(data, UTF_8).split(";");
- long version;
- try {
- version = Long.parseLong(parts[0]) & METADATA_VERSION_MASK;
- } catch (Exception exc) {
- throw new IOException("Invalid ledger entry, "
- + new String(data, UTF_8));
- }
-
- if (!skipMinVersionCheck && version < LogSegmentMetadata.LEDGER_METADATA_OLDEST_SUPPORTED_VERSION) {
- throw new UnsupportedMetadataVersionException("Ledger metadata version '" + version + "' is no longer supported: "
- + new String(data, UTF_8));
- }
-
- if (version > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION) {
- throw new UnsupportedMetadataVersionException("Metadata version '" + version + "' is higher than the highest supported version : "
- + new String(data, UTF_8));
- }
-
- if (LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value == version) {
- return parseDataV1(path, data, parts);
- } else if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value == version) {
- return parseDataV2(path, data, parts);
- } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version &&
- LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version) {
- return parseDataVersionsWithMinActiveDLSN(path, data, parts);
- } else {
- assert(version >= LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
- return parseDataVersionsWithSequenceId(path, data, parts);
- }
- }
-
- public String getFinalisedData() {
- return getFinalisedData(this.version);
- }
-
- public String getFinalisedData(LogSegmentMetadataVersion version) {
- String finalisedData;
- final long logSegmentSeqNo = getLogSegmentSequenceNumber();
- final long lastEntryId = getLastEntryId();
- final long lastSlotId = getLastSlotId();
- final long minActiveEntryId = minActiveDLSN.getEntryId();
- final long minActiveSlotId = minActiveDLSN.getSlotId();
-
- if (LogSegmentMetadataVersion.VERSION_V1_ORIGINAL == version) {
- if (inprogress) {
- finalisedData = String.format("%d;%d;%d",
- version.value, logSegmentId, firstTxId);
- } else {
- long versionAndCount = ((long) version.value) | ((long)recordCount << LOGRECORD_COUNT_SHIFT);
- finalisedData = String.format("%d;%d;%d;%d;%d",
- versionAndCount, logSegmentId, firstTxId, lastTxId, completionTime);
- }
- } else {
- long versionStatusCount = ((long) version.value);
- versionStatusCount |= ((status & METADATA_STATUS_BIT_MAX) << STATUS_BITS_SHIFT);
- versionStatusCount |= (((long) regionId & MAX_REGION_ID) << REGION_SHIFT);
- if (!inprogress) {
- versionStatusCount |= ((long)recordCount << LOGRECORD_COUNT_SHIFT);
- }
- if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO == version) {
- if (inprogress) {
- finalisedData = String.format("%d;%d;%d;%d",
- versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo);
- } else {
- finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d",
- versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
- logSegmentSeqNo, lastEntryId, lastSlotId);
- }
- } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version.value &&
- LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version.value) {
- if (inprogress) {
- finalisedData = String.format("%d;%d;%d;%d;%d;%d",
- versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId);
- } else {
- finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
- versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
- logSegmentSeqNo, lastEntryId, lastSlotId, minActiveEntryId, minActiveSlotId);
- }
- } else if (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version.value &&
- LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version.value) {
- if (inprogress) {
- finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d",
- versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId, startSequenceId);
- } else {
- finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
- versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
- logSegmentSeqNo, lastEntryId, lastSlotId, minActiveEntryId, minActiveSlotId, startSequenceId);
- }
- } else {
- throw new IllegalStateException("Unsupported log segment ledger metadata version '" + version + "'");
- }
- }
- return finalisedData;
- }
-
- String getSegmentName() {
- String[] parts = this.zkPath.split("/");
- if (parts.length <= 0) {
- throw new IllegalStateException("ZK Path is not valid");
- }
- return parts[parts.length - 1];
- }
-
- public void write(ZooKeeperClient zkc)
- throws IOException, KeeperException.NodeExistsException {
- String finalisedData = getFinalisedData(version);
- try {
- zkc.get().create(zkPath, finalisedData.getBytes(UTF_8),
- zkc.getDefaultACL(), CreateMode.PERSISTENT);
- } catch (KeeperException.NodeExistsException nee) {
- throw nee;
- } catch (InterruptedException ie) {
- throw new DLInterruptedException("Interrupted on creating ledger znode " + zkPath, ie);
- } catch (Exception e) {
- LOG.error("Error creating ledger znode {}", zkPath, e);
- throw new IOException("Error creating ledger znode " + zkPath);
- }
- }
-
- boolean checkEquivalence(ZooKeeperClient zkc, String path) {
- try {
- LogSegmentMetadata other = FutureUtils.result(read(zkc, path));
- if (LOG.isTraceEnabled()) {
- LOG.trace("Verifying {} against {}", this, other);
- }
-
- boolean retVal;
-
- // All fields may not be comparable so only compare the ones
- // that can be compared
- // completionTime is set when a node is finalized, so that
- // cannot be compared
- // if the node is inprogress, don't compare the lastTxId either
- if (this.getLogSegmentSequenceNumber() != other.getLogSegmentSequenceNumber() ||
- this.logSegmentId != other.logSegmentId ||
- this.firstTxId != other.firstTxId) {
- retVal = false;
- } else if (this.inprogress) {
- retVal = other.inprogress;
- } else {
- retVal = (!other.inprogress && (this.lastTxId == other.lastTxId));
- }
-
- if (!retVal) {
- LOG.warn("Equivalence check failed between {} and {}", this, other);
- }
-
- return retVal;
- } catch (Exception e) {
- LOG.error("Could not check equivalence between:" + this + " and data in " + path, e);
- return false;
- }
- }
-
- public boolean equals(Object o) {
- if (!(o instanceof LogSegmentMetadata)) {
- return false;
- }
- LogSegmentMetadata ol = (LogSegmentMetadata) o;
- return getLogSegmentSequenceNumber() == ol.getLogSegmentSequenceNumber()
- && logSegmentId == ol.logSegmentId
- && firstTxId == ol.firstTxId
- && lastTxId == ol.lastTxId
- && version == ol.version
- && completionTime == ol.completionTime
- && Objects.equal(lastDLSN, ol.lastDLSN)
- && Objects.equal(minActiveDLSN, ol.minActiveDLSN)
- && startSequenceId == ol.startSequenceId
- && status == ol.status;
- }
-
- public int hashCode() {
- int hash = 1;
- hash = hash * 31 + (int) logSegmentId;
- hash = hash * 31 + (int) firstTxId;
- hash = hash * 31 + (int) lastTxId;
- hash = hash * 31 + version.value;
- hash = hash * 31 + (int) completionTime;
- hash = hash * 31 + (int) getLogSegmentSequenceNumber();
- return hash;
- }
-
- public String toString() {
- return "[LogSegmentId:" + logSegmentId +
- ", firstTxId:" + firstTxId +
- ", lastTxId:" + lastTxId +
- ", version:" + version +
- ", completionTime:" + completionTime +
- ", recordCount:" + recordCount +
- ", regionId:" + regionId +
- ", status:" + status +
- ", logSegmentSequenceNumber:" + getLogSegmentSequenceNumber() +
- ", lastEntryId:" + getLastEntryId() +
- ", lastSlotId:" + getLastSlotId() +
- ", inprogress:" + inprogress +
- ", minActiveDLSN:" + minActiveDLSN +
- ", startSequenceId:" + startSequenceId +
- "]";
- }
-
- public Mutator mutator() {
- return new Mutator(this);
- }
-
-
- //
- // Version Checking Utilities
- //
-
- public boolean supportsLogSegmentSequenceNo() {
- return supportsLogSegmentSequenceNo(version.value);
- }
-
- /**
- * Whether the provided version supports log segment sequence number.
- *
- * @param version
- * log segment metadata version
- * @return true if this log segment supports log segment sequence number.
- */
- public static boolean supportsLogSegmentSequenceNo(int version) {
- return version >= LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value;
- }
-
- /**
- * Whether the provided version supports enveloping entries before writing to bookkeeper.
- *
- * @param version
- * log segment metadata version
- * @return true if this log segment supports enveloping entries
- */
- public static boolean supportsEnvelopedEntries(int version) {
- return version >= LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value;
- }
-
- public boolean supportsSequenceId() {
- return supportsSequenceId(version.value);
- }
-
- /**
- * Whether the provided version supports sequence id.
- *
- * @param version
- * log segment metadata version
- * @return true if the log segment support sequence id.
- */
- public static boolean supportsSequenceId(int version) {
- return version >= LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LogWriter.java
deleted file mode 100644
index d7de586..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogWriter.java
+++ /dev/null
@@ -1,78 +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.io.Abortable;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-
-/*
-* A generic interface class to support writing log records into
-* a persistent distributed log.
-*/
-public interface LogWriter extends Closeable, Abortable {
- /**
- * Write a log record to the stream.
- *
- * @param record single log record
- * @throws IOException
- */
- public void write(LogRecord record) throws IOException;
-
-
- /**
- * Write a list of log records to the stream.
- *
- * @param records list of log records
- * @throws IOException
- */
- @Deprecated
- public int writeBulk(List<LogRecord> records) throws IOException;
-
- /**
- * All data that has been written to the stream so far will be sent to
- * persistent storage.
- * The transmission is asynchronous and new data can be still written to the
- * stream while flushing is performed.
- *
- * TODO: rename this to flush()
- */
- public long setReadyToFlush() throws IOException;
-
- /**
- * Flush and sync all data that is ready to be flush
- * {@link #setReadyToFlush()} into underlying persistent store.
- * @throws IOException
- *
- * TODO: rename this to commit()
- */
- public long flushAndSync() throws IOException;
-
- /**
- * Flushes all the data up to this point,
- * adds the end of stream marker and marks the stream
- * as read-only in the metadata. No appends to the
- * stream will be allowed after this point
- *
- * @throws IOException
- */
- public void markEndOfStream() throws IOException;
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxLogSegmentSequenceNo.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxLogSegmentSequenceNo.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxLogSegmentSequenceNo.java
deleted file mode 100644
index 9bfaaba..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxLogSegmentSequenceNo.java
+++ /dev/null
@@ -1,72 +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.util.DLUtils;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-
-/**
- * Utility class for storing and reading max ledger sequence number
- */
-class MaxLogSegmentSequenceNo {
-
- Version version;
- long maxSeqNo;
-
- MaxLogSegmentSequenceNo(Versioned<byte[]> logSegmentsData) {
- if (null != logSegmentsData
- && null != logSegmentsData.getValue()
- && null != logSegmentsData.getVersion()) {
- version = logSegmentsData.getVersion();
- try {
- maxSeqNo = DLUtils.deserializeLogSegmentSequenceNumber(logSegmentsData.getValue());
- } catch (NumberFormatException nfe) {
- maxSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
- }
- } else {
- maxSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
- if (null != logSegmentsData && null != logSegmentsData.getVersion()) {
- version = logSegmentsData.getVersion();
- } else {
- throw new IllegalStateException("Invalid MaxLogSegmentSequenceNo found - " + logSegmentsData);
- }
- }
- }
-
- synchronized Version getVersion() {
- return version;
- }
-
- synchronized long getSequenceNumber() {
- return maxSeqNo;
- }
-
- synchronized MaxLogSegmentSequenceNo update(Version version, long logSegmentSeqNo) {
- if (version.compare(this.version) == Version.Occurred.AFTER) {
- this.version = version;
- this.maxSeqNo = logSegmentSeqNo;
- }
- return this;
- }
-
- public synchronized Versioned<Long> getVersionedData(long seqNo) {
- return new Versioned<Long>(seqNo, version);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxTxId.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxTxId.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxTxId.java
deleted file mode 100644
index 8eabf88..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxTxId.java
+++ /dev/null
@@ -1,72 +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.util.DLUtils;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Utility class for storing and reading
- * the max seen txid in zookeeper
- */
-class MaxTxId {
- static final Logger LOG = LoggerFactory.getLogger(MaxTxId.class);
-
- private Version version;
- private long currentMax;
-
- MaxTxId(Versioned<byte[]> maxTxIdData) {
- if (null != maxTxIdData
- && null != maxTxIdData.getValue()
- && null != maxTxIdData.getVersion()) {
- this.version = maxTxIdData.getVersion();
- try {
- this.currentMax = DLUtils.deserializeTransactionId(maxTxIdData.getValue());
- } catch (NumberFormatException e) {
- LOG.warn("Invalid txn id stored in {}", e);
- this.currentMax = DistributedLogConstants.INVALID_TXID;
- }
- } else {
- this.currentMax = DistributedLogConstants.INVALID_TXID;
- if (null != maxTxIdData && null != maxTxIdData.getVersion()) {
- this.version = maxTxIdData.getVersion();
- } else {
- throw new IllegalStateException("Invalid MaxTxId found - " + maxTxIdData);
- }
- }
- }
-
- synchronized void update(Version version, long txId) {
- if (version.compare(this.version) == Version.Occurred.AFTER) {
- this.version = version;
- this.currentMax = txId;
- }
- }
-
- synchronized long get() {
- return currentMax;
- }
-
- public synchronized Versioned<Long> getVersionedData(long txId) {
- return new Versioned<Long>(Math.max(txId, get()), version);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/MetadataAccessor.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/MetadataAccessor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/MetadataAccessor.java
deleted file mode 100644
index f6ff587..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/MetadataAccessor.java
+++ /dev/null
@@ -1,43 +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.io.AsyncCloseable;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public interface MetadataAccessor extends Closeable, AsyncCloseable {
- /**
- * Get the name of the stream managed by this log manager
- * @return streamName
- */
- public String getStreamName();
-
- public void createOrUpdateMetadata(byte[] metadata) throws IOException;
-
- public void deleteMetadata() throws IOException;
-
- public byte[] getMetadata() throws IOException;
-
- /**
- * Close the distributed log metadata, freeing any resources it may hold.
- */
- public void close() throws IOException;
-
-}
[18/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
deleted file mode 100644
index 0e5e6d4..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
+++ /dev/null
@@ -1,184 +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.metadata;
-
-import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-
-public class LogSegmentMetadataStoreUpdater implements MetadataUpdater {
-
- static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadataStoreUpdater.class);
-
- public static MetadataUpdater createMetadataUpdater(DistributedLogConfiguration conf,
- LogSegmentMetadataStore metadataStore) {
- return new LogSegmentMetadataStoreUpdater(conf, metadataStore);
- }
-
- protected final LogSegmentMetadataStore metadataStore;
- protected final LogSegmentMetadata.LogSegmentMetadataVersion metadataVersion;
-
- protected LogSegmentMetadataStoreUpdater(DistributedLogConfiguration conf,
- LogSegmentMetadataStore metadataStore) {
- this.metadataStore = metadataStore;
- this.metadataVersion = LogSegmentMetadata.LogSegmentMetadataVersion.of(conf.getDLLedgerMetadataLayoutVersion());
- }
-
- private String formatLogSegmentSequenceNumber(long logSegmentSeqNo) {
- return String.format("%018d", logSegmentSeqNo);
- }
-
- @Override
- public Transaction<Object> transaction() {
- return metadataStore.transaction();
- }
-
- @Override
- public Future<LogSegmentMetadata> updateLastRecord(LogSegmentMetadata segment,
- LogRecordWithDLSN record) {
- DLSN dlsn = record.getDlsn();
- Preconditions.checkState(!segment.isInProgress(),
- "Updating last dlsn for an inprogress log segment isn't supported.");
- Preconditions.checkArgument(segment.isDLSNinThisSegment(dlsn),
- "DLSN " + dlsn + " doesn't belong to segment " + segment);
- final LogSegmentMetadata newSegment = segment.mutator()
- .setLastDLSN(dlsn)
- .setLastTxId(record.getTransactionId())
- .setRecordCount(record)
- .build();
- return updateSegmentMetadata(newSegment);
- }
-
- @Override
- public Future<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
- long logSegmentSeqNo) {
- String newZkPath = segment.getZkPath()
- .replace(formatLogSegmentSequenceNumber(segment.getLogSegmentSequenceNumber()),
- formatLogSegmentSequenceNumber(logSegmentSeqNo));
- final LogSegmentMetadata newSegment = segment.mutator()
- .setLogSegmentSequenceNumber(logSegmentSeqNo)
- .setZkPath(newZkPath)
- .build();
- return addNewSegmentAndDeleteOldSegment(newSegment, segment);
- }
-
- /**
- * Change the truncation status of a <i>log segment</i> to be active
- *
- * @param segment log segment to change truncation status to active.
- * @return new log segment
- */
- @Override
- public Future<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment) {
- final LogSegmentMetadata newSegment = segment.mutator()
- .setTruncationStatus(LogSegmentMetadata.TruncationStatus.ACTIVE)
- .build();
- return addNewSegmentAndDeleteOldSegment(newSegment, segment);
- }
-
- /**
- * Change the truncation status of a <i>log segment</i> to truncated
- *
- * @param segment log segment to change truncation status to truncated.
- * @return new log segment
- */
- @Override
- public Future<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment) {
- final LogSegmentMetadata newSegment = segment.mutator()
- .setTruncationStatus(LogSegmentMetadata.TruncationStatus.TRUNCATED)
- .build();
- return addNewSegmentAndDeleteOldSegment(newSegment, segment);
- }
-
- @Override
- public LogSegmentMetadata setLogSegmentTruncated(Transaction<Object> txn, LogSegmentMetadata segment) {
- final LogSegmentMetadata newSegment = segment.mutator()
- .setTruncationStatus(LogSegmentMetadata.TruncationStatus.TRUNCATED)
- .build();
- addNewSegmentAndDeleteOldSegment(txn, newSegment, segment);
- return newSegment;
- }
-
- /**
- * Change the truncation status of a <i>log segment</i> to partially truncated
- *
- * @param segment log segment to change sequence number.
- * @param minActiveDLSN DLSN within the log segment before which log has been truncated
- * @return new log segment
- */
- @Override
- public Future<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment, DLSN minActiveDLSN) {
- final LogSegmentMetadata newSegment = segment.mutator()
- .setTruncationStatus(LogSegmentMetadata.TruncationStatus.PARTIALLY_TRUNCATED)
- .setMinActiveDLSN(minActiveDLSN)
- .build();
- return addNewSegmentAndDeleteOldSegment(newSegment, segment);
- }
-
- @Override
- public LogSegmentMetadata setLogSegmentPartiallyTruncated(Transaction<Object> txn,
- LogSegmentMetadata segment,
- DLSN minActiveDLSN) {
- final LogSegmentMetadata newSegment = segment.mutator()
- .setTruncationStatus(LogSegmentMetadata.TruncationStatus.PARTIALLY_TRUNCATED)
- .setMinActiveDLSN(minActiveDLSN)
- .build();
- addNewSegmentAndDeleteOldSegment(txn, newSegment, segment);
- return newSegment;
- }
-
- protected Future<LogSegmentMetadata> updateSegmentMetadata(final LogSegmentMetadata segment) {
- Transaction<Object> txn = transaction();
- metadataStore.updateLogSegment(txn, segment);
- return txn.execute().map(new AbstractFunction1<Void, LogSegmentMetadata>() {
- @Override
- public LogSegmentMetadata apply(Void value) {
- return segment;
- }
- });
- }
-
- protected Future<LogSegmentMetadata> addNewSegmentAndDeleteOldSegment(
- final LogSegmentMetadata newSegment, LogSegmentMetadata oldSegment) {
- LOG.info("old segment {} new segment {}", oldSegment, newSegment);
- Transaction<Object> txn = transaction();
- addNewSegmentAndDeleteOldSegment(txn, newSegment, oldSegment);
- return txn.execute().map(new AbstractFunction1<Void, LogSegmentMetadata>() {
- @Override
- public LogSegmentMetadata apply(Void value) {
- return newSegment;
- }
- });
- }
-
- protected void addNewSegmentAndDeleteOldSegment(Transaction<Object> txn,
- LogSegmentMetadata newSegment,
- LogSegmentMetadata oldSegment) {
- metadataStore.deleteLogSegment(txn, oldSegment, null);
- metadataStore.createLogSegment(txn, newSegment, null);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogStreamMetadataStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogStreamMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogStreamMetadataStore.java
deleted file mode 100644
index 7242a5e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogStreamMetadataStore.java
+++ /dev/null
@@ -1,114 +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.metadata;
-
-import com.google.common.annotations.Beta;
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.util.PermitManager;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-
-import java.io.Closeable;
-import java.net.URI;
-
-/**
- * The interface to manage the log stream metadata. The implementation is responsible
- * for creating the metadata layout.
- */
-@Beta
-public interface LogStreamMetadataStore extends Closeable {
-
- /**
- * Create a transaction for the metadata operations happening in the metadata store.
- *
- * @return transaction for the metadata operations
- */
- Transaction<Object> newTransaction();
-
- /**
- * Ensure the existence of a log stream
- *
- * @param uri the location of the log stream
- * @param logName the name of the log stream
- * @return future represents the existence of a log stream. {@link com.twitter.distributedlog.LogNotFoundException}
- * is thrown if the log doesn't exist
- */
- Future<Void> logExists(URI uri, String logName);
-
- /**
- * Create the read lock for the log stream.
- *
- * @param metadata the metadata for a log stream
- * @param readerId the reader id used for lock
- * @return the read lock
- */
- Future<DistributedLock> createReadLock(LogMetadataForReader metadata,
- Optional<String> readerId);
-
- /**
- * Create the write lock for the log stream.
- *
- * @param metadata the metadata for a log stream
- * @return the write lock
- */
- DistributedLock createWriteLock(LogMetadataForWriter metadata);
-
- /**
- * Create the metadata of a log.
- *
- * @param uri the location to store the metadata of the log
- * @param streamName the name of the log stream
- * @param ownAllocator whether to use its own allocator or external allocator
- * @param createIfNotExists flag to create the stream if it doesn't exist
- * @return the metadata of the log
- */
- Future<LogMetadataForWriter> getLog(URI uri,
- String streamName,
- boolean ownAllocator,
- boolean createIfNotExists);
-
- /**
- * Delete the metadata of a log.
- *
- * @param uri the location to store the metadata of the log
- * @param streamName the name of the log stream
- * @return future represents the result of the deletion.
- */
- Future<Void> deleteLog(URI uri, String streamName);
-
- /**
- * Get the log segment metadata store.
- *
- * @return the log segment metadata store.
- */
- LogSegmentMetadataStore getLogSegmentMetadataStore();
-
- /**
- * Get the permit manager for this metadata store. It can be used for limiting the concurrent
- * metadata operations. The implementation can disable handing over the permits when the metadata
- * store is unavailable (for example zookeeper session expired).
- *
- * @return the permit manager
- */
- PermitManager getPermitManager();
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataResolver.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataResolver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataResolver.java
deleted file mode 100644
index 417cab8..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataResolver.java
+++ /dev/null
@@ -1,51 +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.metadata;
-
-import java.io.IOException;
-import java.net.URI;
-
-/**
- * Resolver to resolve the metadata used to instantiate a DL instance.
- *
- * <p>
- * E.g. we stored a common dl config under /messaging/distributedlog to use
- * bookkeeper cluster x. so all the distributedlog instances under this path
- * inherit this dl config. if a dl D is allocated under /messaging/distributedlog,
- * but use a different cluster y, so its metadata is stored /messaging/distributedlog/D.
- * The resolver resolve the URI
- * </p>
- *
- * <p>
- * The resolver looks up the uri path and tries to interpret the path segments from
- * bottom-to-top to see if there is a DL metadata bound. It stops when it found valid
- * dl metadata.
- * </p>
- */
-public interface MetadataResolver {
-
- /**
- * Resolve the path to get the DL metadata.
- *
- * @param uri
- * dl uri
- * @return dl metadata.
- * @throws IOException
- */
- public DLMetadata resolve(URI uri) throws IOException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataUpdater.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataUpdater.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataUpdater.java
deleted file mode 100644
index b98f168..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataUpdater.java
+++ /dev/null
@@ -1,120 +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.metadata;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-
-/**
- * An updater to update metadata. It contains utility functions on mutating metadata.
- */
-public interface MetadataUpdater {
-
- /**
- * Start a transaction on metadata updates
- *
- * @return transaction
- */
- Transaction<Object> transaction();
-
- /**
- * Update the log segment metadata with correct last <i>record</i>.
- *
- * @param segment
- * log segment to update last dlsn.
- * @param record
- * correct last record.
- * @return new log segment
- */
- Future<LogSegmentMetadata> updateLastRecord(LogSegmentMetadata segment,
- LogRecordWithDLSN record);
-
- /**
- * Change ledger sequence number of <i>segment</i> to given <i>logSegmentSeqNo</i>.
- *
- * @param segment
- * log segment to change sequence number.
- * @param logSegmentSeqNo
- * ledger sequence number to change.
- * @return new log segment
- */
- Future<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
- long logSegmentSeqNo);
-
- /**
- * Change the truncation status of a <i>log segment</i> to be active
- *
- * @param segment
- * log segment to change truncation status to active.
- * @return new log segment
- */
- Future<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment);
-
- /**
- * Change the truncation status of a <i>log segment</i> to truncated
- *
- * @param segment
- * log segment to change truncation status to truncated.
- * @return new log segment
- */
- Future<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment);
-
- /**
- * Change the truncation status of a <i>log segment</i> to truncated. The operation won't be executed
- * immediately. The update only happens after {@link Transaction#execute()}.
- *
- * @param txn
- * transaction used to set the log segment status
- * @param segment
- * segment to set truncation status to truncated
- * @return log segment that truncation status is set to truncated.
- */
- LogSegmentMetadata setLogSegmentTruncated(Transaction<Object> txn, LogSegmentMetadata segment);
-
- /**
- * Change the truncation status of a <i>log segment</i> to partially truncated
- *
- * @param segment
- * log segment to change sequence number.
- * @param minActiveDLSN
- * DLSN within the log segment before which log has been truncated
- * @return new log segment
- */
- Future<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment,
- DLSN minActiveDLSN);
-
- /**
- * Change the truncation status of a <i>log segment</i> to partially truncated. The operation won't be
- * executed until {@link Transaction#execute()}.
- *
- * @param txn
- * transaction used to set the log segment status
- * @param segment
- * segment to set truncation status to partially truncated
- * @param minActiveDLSN
- * DLSN within the log segment before which log has been truncated
- * @return log segment that truncation status has been set to partially truncated
- */
- LogSegmentMetadata setLogSegmentPartiallyTruncated(Transaction<Object> txn,
- LogSegmentMetadata segment,
- DLSN minActiveDLSN);
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/package-info.java
deleted file mode 100644
index f740c77..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Metadata management for distributedlog.
- */
-package com.twitter.distributedlog.metadata;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 5d1d888..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
+++ /dev/null
@@ -1,190 +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.namespace;
-
-import com.google.common.annotations.Beta;
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-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 java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * A namespace is the basic unit for managing a set of distributedlogs.
- *
- * <h4>Namespace Interface</h4>
- *
- * <P>
- * The <code>DistributedLogNamespace</code> interface is implemented by different backend providers.
- * There are several components are required for an implementation:
- * <OL>
- * <LI>Log Management -- manage logs in a given namespace. e.g. create/open/delete log, list of logs,
- * watch the changes of logs.
- * <LI>Access Control -- manage the access controls for logs in the namespace.
- * </OL>
- * </P>
- *
- * <h4>Namespace Location</h4>
- *
- * At the highest level, a <code>DistributedLogNamespace</code> is located by a <code>URI</code>. The location
- * URI is in string form has the syntax
- *
- * <blockquote>
- * distributedlog[<tt><b>-</b></tt><i>provider</i>]<tt><b>:</b></tt><i>provider-specific-path</i>
- * </blockquote>
- *
- * where square brackets [...] delineate optional components and the characters <tt><b>-</b></tt> and <tt><b>:</b></tt>
- * stand for themselves.
- *
- * The <code>provider</code> part in the URI indicates what is the backend used for this namespace. For example:
- * <i>distributedlog-bk</i> URI is storing logs in bookkeeper, while <i>distributedlog-mem</i> URI is storing logs in
- * memory. The <code>provider</code> part is optional. It would use bookkeeper backend if the <i>provider</i> part
- * is omitted.
- *
- * @see DistributedLogManager
- * @since 0.3.32
- */
-@Beta
-public interface DistributedLogNamespace {
-
- /**
- * Get the namespace driver used by this namespace.
- *
- * @return namespace driver
- */
- NamespaceDriver getNamespaceDriver();
-
- //
- // Method to operate logs
- //
-
- /**
- * Create a log named <i>logName</i>.
- *
- * @param logName
- * name of the log
- * @throws InvalidStreamNameException if log name is invalid.
- * @throws IOException when encountered issues with backend.
- */
- void createLog(String logName)
- throws InvalidStreamNameException, IOException;
-
- /**
- * Delete a log named <i>logName</i>.
- *
- * @param logName
- * name of the log
- * @throws InvalidStreamNameException if log name is invalid
- * @throws LogNotFoundException if log doesn't exist
- * @throws IOException when encountered issues with backend
- */
- void deleteLog(String logName)
- throws InvalidStreamNameException, LogNotFoundException, IOException;
-
- /**
- * Open a log named <i>logName</i>.
- * A distributedlog manager is returned to access log <i>logName</i>.
- *
- * @param logName
- * name of the log
- * @return distributedlog manager instance.
- * @throws InvalidStreamNameException if log name is invalid.
- * @throws IOException when encountered issues with backend.
- */
- DistributedLogManager openLog(String logName)
- throws InvalidStreamNameException, IOException;
-
- /**
- * Open a log named <i>logName</i> with specific log configurations.
- *
- * <p>This method allows the caller to override global configuration settings by
- * supplying log configuration overrides. Log config overrides come in two flavors,
- * static and dynamic. Static config never changes in the lifecyle of <code>DistributedLogManager</code>,
- * dynamic config changes by reloading periodically and safe to access from any context.</p>
- *
- * @param logName
- * name of the log
- * @param logConf
- * static log configuration
- * @param dynamicLogConf
- * dynamic log configuration
- * @return distributedlog manager instance.
- * @throws InvalidStreamNameException if log name is invalid.
- * @throws IOException when encountered issues with backend.
- */
- DistributedLogManager openLog(String logName,
- Optional<DistributedLogConfiguration> logConf,
- Optional<DynamicDistributedLogConfiguration> dynamicLogConf,
- Optional<StatsLogger> perStreamStatsLogger)
- throws InvalidStreamNameException, IOException;
-
- /**
- * Check whether the log <i>logName</i> exist.
- *
- * @param logName
- * name of the log
- * @return <code>true</code> if the log exists, otherwise <code>false</code>.
- * @throws IOException when encountered exceptions on checking
- */
- boolean logExists(String logName)
- throws IOException;
-
- /**
- * Retrieve the logs under the namespace.
- *
- * @return iterator of the logs under the namespace.
- * @throws IOException when encountered issues with backend.
- */
- Iterator<String> getLogs()
- throws IOException;
-
- //
- // Methods for namespace
- //
-
- /**
- * Register namespace listener on stream updates under the namespace.
- *
- * @param listener
- * listener to receive stream updates under the namespace
- */
- void registerNamespaceListener(NamespaceListener listener);
-
- /**
- * Create an access control manager to manage/check acl for logs.
- *
- * @return access control manager for logs under the namespace.
- * @throws IOException
- */
- AccessControlManager createAccessControlManager()
- throws IOException;
-
- /**
- * Close the namespace.
- */
- void close();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 07b3848..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
+++ /dev/null
@@ -1,278 +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.namespace;
-
-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.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-
-/**
- * Builder to construct a <code>DistributedLogNamespace</code>.
- * The builder takes the responsibility of loading backend according to the uri.
- *
- * @see DistributedLogNamespace
- * @since 0.3.32
- */
-public class DistributedLogNamespaceBuilder {
-
- private static final Logger logger = LoggerFactory.getLogger(DistributedLogNamespaceBuilder.class);
-
- public static DistributedLogNamespaceBuilder newBuilder() {
- return new DistributedLogNamespaceBuilder();
- }
-
- private DistributedLogConfiguration _conf = null;
- private DynamicDistributedLogConfiguration _dynConf = null;
- private URI _uri = null;
- private StatsLogger _statsLogger = NullStatsLogger.INSTANCE;
- private StatsLogger _perLogStatsLogger = NullStatsLogger.INSTANCE;
- private FeatureProvider _featureProvider = null;
- private String _clientId = DistributedLogConstants.UNKNOWN_CLIENT_ID;
- private int _regionId = DistributedLogConstants.LOCAL_REGION_ID;
-
- // private constructor
- private DistributedLogNamespaceBuilder() {}
-
- /**
- * DistributedLog Configuration used for the namespace.
- *
- * @param conf
- * distributedlog configuration
- * @return namespace builder.
- */
- public DistributedLogNamespaceBuilder conf(DistributedLogConfiguration conf) {
- this._conf = conf;
- return this;
- }
-
- /**
- * 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
- * namespace location uri.
- * @see DistributedLogNamespace
- * @return namespace builder.
- */
- public DistributedLogNamespaceBuilder uri(URI uri) {
- this._uri = uri;
- return this;
- }
-
- /**
- * Stats Logger used for stats collection
- *
- * @param statsLogger
- * stats logger
- * @return namespace builder.
- */
- public DistributedLogNamespaceBuilder statsLogger(StatsLogger statsLogger) {
- this._statsLogger = statsLogger;
- return this;
- }
-
- /**
- * Stats Logger used for collecting per log stats.
- *
- * @param statsLogger
- * stats logger for collecting per log stats
- * @return namespace builder.
- */
- public DistributedLogNamespaceBuilder perLogStatsLogger(StatsLogger statsLogger) {
- this._perLogStatsLogger = statsLogger;
- return this;
- }
-
- /**
- * Feature provider used to control the availabilities of features in the namespace.
- *
- * @param featureProvider
- * feature provider to control availabilities of features.
- * @return namespace builder.
- */
- public DistributedLogNamespaceBuilder featureProvider(FeatureProvider featureProvider) {
- this._featureProvider = featureProvider;
- return this;
- }
-
- /**
- * Client Id used for accessing the namespace
- *
- * @param clientId
- * client id used for accessing the namespace
- * @return namespace builder.
- */
- public DistributedLogNamespaceBuilder clientId(String clientId) {
- this._clientId = clientId;
- return this;
- }
-
- /**
- * Region Id used for encoding logs in the namespace. The region id
- * is useful when the namespace is globally spanning over regions.
- *
- * @param regionId
- * region id.
- * @return namespace builder.
- */
- public DistributedLogNamespaceBuilder regionId(int regionId) {
- this._regionId = regionId;
- 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.
- *
- * @return the namespace instance.
- * @throws IllegalArgumentException when there is illegal argument provided in the builder
- * @throws NullPointerException when there is null argument provided in the builder
- * @throws IOException when fail to build the backend
- */
- public DistributedLogNamespace build()
- throws IllegalArgumentException, NullPointerException, IOException {
- // Check arguments
- Preconditions.checkNotNull(_conf, "No DistributedLog Configuration.");
- Preconditions.checkNotNull(_uri, "No DistributedLog URI");
-
- // validate the configuration
- _conf.validate();
- if (null == _dynConf) {
- _dynConf = ConfUtils.getConstDynConf(_conf);
- }
-
- // 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);
- logger.info("No feature provider is set. All features are disabled now.");
- } else {
- featureProvider = _featureProvider;
- }
-
- // 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 new BKDistributedLogNamespace(
- _conf,
- normalizedUri,
- driver,
- scheduler,
- featureProvider,
- writeLimiter,
- failureInjector,
- _statsLogger,
- perLogStatsLogger,
- DLUtils.normalizeClientId(_clientId),
- _regionId);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 738f124..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java
+++ /dev/null
@@ -1,138 +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.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);
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 79945ad..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java
+++ /dev/null
@@ -1,180 +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.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);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceWatcher.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceWatcher.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceWatcher.java
deleted file mode 100644
index f836520..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceWatcher.java
+++ /dev/null
@@ -1,61 +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.namespace;
-
-import com.twitter.distributedlog.callback.NamespaceListener;
-
-import java.util.concurrent.CopyOnWriteArraySet;
-
-/**
- * Namespace Watcher watching namespace changes.
- */
-public abstract class NamespaceWatcher {
-
- protected final CopyOnWriteArraySet<NamespaceListener> listeners =
- new CopyOnWriteArraySet<NamespaceListener>();
-
- /**
- * Register listener for namespace changes.
- *
- * @param listener
- * listener to add
- */
- public void registerListener(NamespaceListener listener) {
- if (listeners.add(listener)) {
- watchNamespaceChanges();
- }
- }
-
- /**
- * Unregister listener from the namespace watcher.
- *
- * @param listener
- * listener to remove from namespace watcher
- */
- public void unregisterListener(NamespaceListener listener) {
- listeners.remove(listener);
- }
-
- /**
- * Watch the namespace changes. It would be triggered each time
- * a namspace listener is added. The implementation should handle
- * this.
- */
- protected abstract void watchNamespaceChanges();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/package-info.java
deleted file mode 100644
index d659f44..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Namespace
- */
-package com.twitter.distributedlog.namespace;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolver.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolver.java
deleted file mode 100644
index 2298faf..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolver.java
+++ /dev/null
@@ -1,113 +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.net;
-
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Abstract DNS resolver for bookkeeper ensemble placement.
- */
-public abstract class DNSResolver implements DNSToSwitchMapping {
- static final Logger LOG = LoggerFactory.getLogger(DNSResolver.class);
-
- protected final ConcurrentMap<String, String> domainNameToNetworkLocation =
- new ConcurrentHashMap<String, String>();
-
- protected final ConcurrentMap<String, String> hostNameToRegion =
- new ConcurrentHashMap<String, String>();
-
- /**
- * Construct the default dns resolver without host-region overrides.
- */
- public DNSResolver() {
- this("");
- }
-
- /**
- * Construct the dns resolver with host-region overrides.
- * <p>
- * <i>hostRegionOverrides</i> is a string of pairs of host-region mapping
- * (host:region) separated by ';'. during dns resolution, the host will be resolved
- * to override region. example: <i>host1:region1;host2:region2;...</i>
- *
- * @param hostRegionOverrides
- * pairs of host-region mapping separated by ';'
- */
- public DNSResolver(String hostRegionOverrides) {
- if (StringUtils.isNotBlank(hostRegionOverrides)) {
- // Host Region Overrides are of the form
- // HN1:R1;HN2:R2;...
- String[] overrides = hostRegionOverrides.split(";");
-
- for (String override : overrides) {
- String[] parts = override.split(":");
- if (parts.length != 2) {
- LOG.warn("Incorrect override specified", override);
- } else {
- hostNameToRegion.putIfAbsent(parts[0], parts[1]);
- }
- }
- } // otherwise, no overrides were specified
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public List<String> resolve(List<String> names) {
- List<String> networkLocations = new ArrayList<String>(names.size());
- for (String name : names) {
- networkLocations.add(resolve(name));
- }
- return networkLocations;
- }
-
- private String resolve(String domainName) {
- String networkLocation = domainNameToNetworkLocation.get(domainName);
- if (null == networkLocation) {
- networkLocation = resolveToNetworkLocation(domainName);
- domainNameToNetworkLocation.put(domainName, networkLocation);
- }
- return networkLocation;
- }
-
- /**
- * Resolve the <code>domainName</code> to its network location.
- *
- * @param domainName
- * domain name
- * @return the network location of <i>domainName</i>
- */
- protected abstract String resolveToNetworkLocation(String domainName);
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void reloadCachedMappings() {
- domainNameToNetworkLocation.clear();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRacks.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRacks.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRacks.java
deleted file mode 100644
index a0298d0..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRacks.java
+++ /dev/null
@@ -1,66 +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.net;
-
-/**
- * Resolve the dns by racks.
- * <p>
- * It resolves domain name like `(region)-(rack)-xxx-xxx.*` to network location
- * `/(region)/(rack)`. If resolution failed, it returns `/default-region/default-rack`.
- * <p>
- * region could be override in <code>hostRegionOverrides</code>. for example, if the
- * host name is <i>regionA-rack1-xx-yyy</i>, it would be resolved to `/regionA/rack1`
- * without any overrides. If the specified overrides is <i>regionA-rack1-xx-yyy:regionB</i>,
- * the resolved network location would be <i>/regionB/rack1</i>.
- * <p>
- * Region overrides provide optimization hits to bookkeeper if two `logical` regions are
- * in same or close locations.
- *
- * @see DNSResolver#DNSResolver(String)
- */
-public class DNSResolverForRacks extends DNSResolver {
- static final String DEFAULT_RACK = "/default-region/default-rack";
-
- public DNSResolverForRacks() {
- }
-
- public DNSResolverForRacks(String hostRegionOverrides) {
- super(hostRegionOverrides);
- }
-
- @Override
- protected String resolveToNetworkLocation(String domainName) {
- String[] parts = domainName.split("\\.");
- if (parts.length <= 0) {
- return DEFAULT_RACK;
- }
-
- String hostName = parts[0];
- String[] labels = hostName.split("-");
- if (labels.length != 4) {
- return DEFAULT_RACK;
- }
-
- String region = hostNameToRegion.get(hostName);
- if (null == region) {
- region = labels[0];
- }
-
- return String.format("/%s/%s", region, labels[1]);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRows.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRows.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRows.java
deleted file mode 100644
index f585640..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRows.java
+++ /dev/null
@@ -1,72 +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.net;
-
-/**
- * Resolve the dns by rows.
- * <p>
- * It resolves domain name like `(region)-(row)xx-xxx-xxx.*` to network location
- * `/(region)/(row)`. If resolution failed, it returns `/default-region/default-row`.
- * <p>
- * region could be override in <code>hostRegionOverrides</code>. for example, if the
- * host name is <i>regionA-row1-xx-yyy</i>, it would be resolved to `/regionA/row1`
- * without any overrides. If the specified overrides is <i>regionA-row1-xx-yyy:regionB</i>,
- * the resolved network location would be <i>/regionB/row1</i>.
- * <p>
- * Region overrides provide optimization hits to bookkeeper if two `logical` regions are
- * in same or close locations.
- *
- * @see DNSResolver#DNSResolver(String)
- */
-public class DNSResolverForRows extends DNSResolver {
- static final String DEFAULT_ROW = "/default-region/default-row";
-
- public DNSResolverForRows() {
- }
-
- public DNSResolverForRows(String hostRegionOverrides) {
- super(hostRegionOverrides);
- }
-
- @Override
- protected String resolveToNetworkLocation(String domainName) {
- String[] parts = domainName.split("\\.");
- if (parts.length <= 0) {
- return DEFAULT_ROW;
- }
- String hostName = parts[0];
- String[] labels = hostName.split("-");
- if (labels.length != 4) {
- return DEFAULT_ROW;
- }
-
- String region = hostNameToRegion.get(hostName);
- if (null == region) {
- region = labels[0];
- }
-
- final String rack = labels[1];
-
- if (rack.length() < 2) {
- // Default to rack name if the rack name format cannot be recognized
- return String.format("/%s/%s", region, rack);
- } else {
- return String.format("/%s/%s", region, rack.substring(0, 2));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/NetUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/NetUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/NetUtils.java
deleted file mode 100644
index ce0d360..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/NetUtils.java
+++ /dev/null
@@ -1,74 +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.net;
-
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-
-/**
- * Utils about network
- */
-public class NetUtils {
-
- /**
- * Get the dns resolver from class <code>resolverClassName</code> with optional
- * <code>hostRegionOverrides</code>.
- * <p>
- * It would try to load the class with the constructor with <code>hostRegionOverrides</code>.
- * If it fails, it would fall back to load the class with default empty constructor.
- * The interpretion of <code>hostRegionOverrides</code> is up to the implementation.
- *
- * @param resolverCls
- * resolver class
- * @param hostRegionOverrides
- * host region overrides
- * @return dns resolver
- */
- public static DNSToSwitchMapping getDNSResolver(Class<? extends DNSToSwitchMapping> resolverCls,
- String hostRegionOverrides) {
- // first try to construct the dns resolver with overrides
- Constructor<? extends DNSToSwitchMapping> constructor;
- Object[] parameters;
- try {
- constructor = resolverCls.getDeclaredConstructor(String.class);
- parameters = new Object[] { hostRegionOverrides };
- } catch (NoSuchMethodException nsme) {
- // no constructor with overrides
- try {
- constructor = resolverCls.getDeclaredConstructor();
- parameters = new Object[0];
- } catch (NoSuchMethodException nsme1) {
- throw new RuntimeException("Unable to find constructor for dns resolver "
- + resolverCls, nsme1);
- }
- }
- constructor.setAccessible(true);
- try {
- return constructor.newInstance(parameters);
- } catch (InstantiationException ie) {
- throw new RuntimeException("Unable to instantiate dns resolver " + resolverCls, ie);
- } catch (IllegalAccessException iae) {
- throw new RuntimeException("Illegal access to dns resolver " + resolverCls, iae);
- } catch (InvocationTargetException ite) {
- throw new RuntimeException("Unable to construct dns resolver " + resolverCls, ite);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/package-info.java
deleted file mode 100644
index 9093fef..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/package-info.java
+++ /dev/null
@@ -1,33 +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.
- */
-/**
- * This package contains all the utilities of network.
- *
- * <h2>DNSResolver</h2>
- *
- * DNS resolver is the utility to resolve host name to a string which represents this host's network location.
- * BookKeeper will use such network locations to place ensemble to ensure rack or region diversity to ensure
- * data availability in the case of switch/router/region is down.
- * <p>
- * Available dns resolvers:
- * <ul>
- * <li>{@link com.twitter.distributedlog.net.DNSResolverForRacks}
- * <li>{@link com.twitter.distributedlog.net.DNSResolverForRows}
- * </ul>
- */
-package com.twitter.distributedlog.net;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/package-info.java
deleted file mode 100644
index 4c1fe57..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Protocol & Core of DistributedLog
- */
-package com.twitter.distributedlog;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRate.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRate.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRate.java
deleted file mode 100644
index 98eae00..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRate.java
+++ /dev/null
@@ -1,24 +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.rate;
-
-public interface MovingAverageRate {
- double get();
- void add(long amount);
- void inc();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRateFactory.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRateFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRateFactory.java
deleted file mode 100644
index a77f753..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRateFactory.java
+++ /dev/null
@@ -1,65 +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.rate;
-
-import com.twitter.util.Duration;
-import com.twitter.util.Function0;
-import com.twitter.util.TimerTask;
-import com.twitter.util.Timer;
-import com.twitter.util.Time;
-import java.util.concurrent.CopyOnWriteArrayList;
-import scala.runtime.BoxedUnit;
-
-public class MovingAverageRateFactory {
-
- private static final int DEFAULT_INTERVAL_SECS = 1;
-
- private final Timer timer;
- private final TimerTask timerTask;
- private final CopyOnWriteArrayList<SampledMovingAverageRate> avgs;
-
- public MovingAverageRateFactory(Timer timer) {
- this.avgs = new CopyOnWriteArrayList<SampledMovingAverageRate>();
- this.timer = timer;
- Function0<BoxedUnit> sampleTask = new Function0<BoxedUnit>() {
- public BoxedUnit apply() {
- sampleAll();
- return null;
- }
- };
- this.timerTask = timer.schedulePeriodically(
- Time.now(), Duration.fromSeconds(DEFAULT_INTERVAL_SECS), sampleTask);
- }
-
- public MovingAverageRate create(int intervalSecs) {
- SampledMovingAverageRate avg = new SampledMovingAverageRate(intervalSecs);
- avgs.add(avg);
- return avg;
- }
-
- public void close() {
- timerTask.cancel();
- avgs.clear();
- }
-
- private void sampleAll() {
- for (SampledMovingAverageRate avg : avgs) {
- avg.sample();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/SampledMovingAverageRate.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/SampledMovingAverageRate.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/SampledMovingAverageRate.java
deleted file mode 100644
index a616324..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/SampledMovingAverageRate.java
+++ /dev/null
@@ -1,58 +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.rate;
-
-import com.twitter.common.stats.Rate;
-import com.twitter.util.TimerTask;
-import com.twitter.util.Timer;
-import com.twitter.util.Time;
-import java.util.concurrent.atomic.AtomicLong;
-
-class SampledMovingAverageRate implements MovingAverageRate {
- private final Rate rate;
- private final AtomicLong total;
-
- private double value;
-
- public SampledMovingAverageRate(int intervalSecs) {
- this.total = new AtomicLong(0);
- this.rate = Rate.of("Ignore", total)
- .withWindowSize(intervalSecs)
- .build();
- this.value = 0;
- }
-
- @Override
- public double get() {
- return value;
- }
-
- @Override
- public void add(long amount) {
- total.getAndAdd(amount);
- }
-
- @Override
- public void inc() {
- add(1);
- }
-
- void sample() {
- value = rate.doSample();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/readahead/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/readahead/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/readahead/package-info.java
deleted file mode 100644
index 4945133..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/readahead/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * ReadAhead Mechanism for distributedlog streaming reads
- */
-package com.twitter.distributedlog.readahead;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstDLSNNotLessThanSelector.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstDLSNNotLessThanSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstDLSNNotLessThanSelector.java
deleted file mode 100644
index 443c503..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstDLSNNotLessThanSelector.java
+++ /dev/null
@@ -1,46 +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.selector;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Save the first record with a dlsn not less than the dlsn provided.
- */
-public class FirstDLSNNotLessThanSelector implements LogRecordSelector {
-
- LogRecordWithDLSN result;
- final DLSN dlsn;
-
- public FirstDLSNNotLessThanSelector(DLSN dlsn) {
- this.dlsn = dlsn;
- }
-
- @Override
- public void process(LogRecordWithDLSN record) {
- if ((record.getDlsn().compareTo(dlsn) >= 0) && (null == result)) {
- this.result = record;
- }
- }
-
- @Override
- public LogRecordWithDLSN result() {
- return this.result;
- }
-}
[20/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKDistributedLock.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKDistributedLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKDistributedLock.java
deleted file mode 100644
index 4d0de7f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKDistributedLock.java
+++ /dev/null
@@ -1,537 +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.lock;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.twitter.concurrent.AsyncSemaphore;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.FutureUtils.OrderedFutureEventListener;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Distributed lock, using ZooKeeper.
- * <p/>
- * The lock is vulnerable to timing issues. For example, the process could
- * encounter a really long GC cycle between acquiring the lock, and writing to
- * a ledger. This could have timed out the lock, and another process could have
- * acquired the lock and started writing to bookkeeper. Therefore other
- * mechanisms are required to ensure correctness (i.e. Fencing).
- * <p/>
- * The lock is only allowed to acquire once. If the lock is acquired successfully,
- * the caller holds the ownership until it loses the ownership either because of
- * others already acquired the lock when session expired or explicitly close it.
- * <p>
- * The caller could use {@link #checkOwnership()} or {@link #checkOwnershipAndReacquire()}
- * to check if it still holds the lock. If it doesn't hold the lock, the caller should
- * give up the ownership and close the lock.
- * <h3>Metrics</h3>
- * All the lock related stats are exposed under `lock`.
- * <ul>
- * <li>lock/acquire: opstats. latency spent on acquiring a lock.
- * <li>lock/reacquire: opstats. latency spent on re-acquiring a lock.
- * <li>lock/internalTryRetries: counter. the number of retries on re-creating internal locks.
- * </ul>
- * Other internal lock related stats are also exposed under `lock`. See {@link SessionLock}
- * for details.
- */
-public class ZKDistributedLock implements LockListener, DistributedLock {
-
- static final Logger LOG = LoggerFactory.getLogger(ZKDistributedLock.class);
-
- private final SessionLockFactory lockFactory;
- private final OrderedScheduler lockStateExecutor;
- private final String lockPath;
- private final long lockTimeout;
- private final DistributedLockContext lockContext = new DistributedLockContext();
-
- private final AsyncSemaphore lockSemaphore = new AsyncSemaphore(1);
- // We have two lock acquire futures:
- // 1. lock acquire future: for the initial acquire op
- // 2. lock reacquire future: for reacquire necessary when session expires, lock is closed
- private Future<ZKDistributedLock> lockAcquireFuture = null;
- private Future<ZKDistributedLock> lockReacquireFuture = null;
- // following variable tracking the status of acquire process
- // => create (internalLock) => tryLock (tryLockFuture) => waitForAcquire (lockWaiter)
- private SessionLock internalLock = null;
- private Future<LockWaiter> tryLockFuture = null;
- private LockWaiter lockWaiter = null;
- // exception indicating if the reacquire failed
- private LockingException lockReacquireException = null;
- // closeFuture
- private volatile boolean closed = false;
- private Future<Void> closeFuture = null;
-
- // A counter to track how many re-acquires happened during a lock's life cycle.
- private final AtomicInteger reacquireCount = new AtomicInteger(0);
- private final StatsLogger lockStatsLogger;
- private final OpStatsLogger acquireStats;
- private final OpStatsLogger reacquireStats;
- private final Counter internalTryRetries;
-
- public ZKDistributedLock(
- OrderedScheduler lockStateExecutor,
- SessionLockFactory lockFactory,
- String lockPath,
- long lockTimeout,
- StatsLogger statsLogger) {
- this.lockStateExecutor = lockStateExecutor;
- this.lockPath = lockPath;
- this.lockTimeout = lockTimeout;
- this.lockFactory = lockFactory;
-
- lockStatsLogger = statsLogger.scope("lock");
- acquireStats = lockStatsLogger.getOpStatsLogger("acquire");
- reacquireStats = lockStatsLogger.getOpStatsLogger("reacquire");
- internalTryRetries = lockStatsLogger.getCounter("internalTryRetries");
- }
-
- private LockClosedException newLockClosedException() {
- return new LockClosedException(lockPath, "Lock is already closed");
- }
-
- private synchronized void checkLockState() throws LockingException {
- if (closed) {
- throw newLockClosedException();
- }
- if (null != lockReacquireException) {
- throw lockReacquireException;
- }
- }
-
- /**
- * Asynchronously acquire the lock. Technically the try phase of this operation--which adds us to the waiter
- * list--is executed synchronously, but the lock wait itself doesn't block.
- */
- public synchronized Future<ZKDistributedLock> asyncAcquire() {
- if (null != lockAcquireFuture) {
- return Future.exception(new UnexpectedException("Someone is already acquiring/acquired lock " + lockPath));
- }
- final Promise<ZKDistributedLock> promise =
- new Promise<ZKDistributedLock>(new Function<Throwable, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Throwable cause) {
- lockStateExecutor.submit(lockPath, new Runnable() {
- @Override
- public void run() {
- asyncClose();
- }
- });
- return BoxedUnit.UNIT;
- }
- });
- final Stopwatch stopwatch = Stopwatch.createStarted();
- promise.addEventListener(new FutureEventListener<ZKDistributedLock>() {
- @Override
- public void onSuccess(ZKDistributedLock lock) {
- acquireStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- }
- @Override
- public void onFailure(Throwable cause) {
- acquireStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- // release the lock if fail to acquire
- asyncClose();
- }
- });
- this.lockAcquireFuture = promise;
- lockStateExecutor.submit(lockPath, new Runnable() {
- @Override
- public void run() {
- doAsyncAcquireWithSemaphore(promise, lockTimeout);
- }
- });
- return promise;
- }
-
- void doAsyncAcquireWithSemaphore(final Promise<ZKDistributedLock> acquirePromise,
- final long lockTimeout) {
- lockSemaphore.acquireAndRun(new AbstractFunction0<Future<ZKDistributedLock>>() {
- @Override
- public Future<ZKDistributedLock> apply() {
- doAsyncAcquire(acquirePromise, lockTimeout);
- return acquirePromise;
- }
- });
- }
-
- void doAsyncAcquire(final Promise<ZKDistributedLock> acquirePromise,
- final long lockTimeout) {
- LOG.trace("Async Lock Acquire {}", lockPath);
- try {
- checkLockState();
- } catch (IOException ioe) {
- FutureUtils.setException(acquirePromise, ioe);
- return;
- }
-
- if (haveLock()) {
- // it already hold the lock
- FutureUtils.setValue(acquirePromise, this);
- return;
- }
-
- lockFactory.createLock(lockPath, lockContext).addEventListener(OrderedFutureEventListener.of(
- new FutureEventListener<SessionLock>() {
- @Override
- public void onSuccess(SessionLock lock) {
- synchronized (ZKDistributedLock.this) {
- if (closed) {
- LOG.info("Skipping tryLocking lock {} since it is already closed", lockPath);
- FutureUtils.setException(acquirePromise, newLockClosedException());
- return;
- }
- }
- synchronized (ZKDistributedLock.this) {
- internalLock = lock;
- internalLock.setLockListener(ZKDistributedLock.this);
- }
- asyncTryLock(lock, acquirePromise, lockTimeout);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(acquirePromise, cause);
- }
- }, lockStateExecutor, lockPath));
- }
-
- void asyncTryLock(SessionLock lock,
- final Promise<ZKDistributedLock> acquirePromise,
- final long lockTimeout) {
- if (null != tryLockFuture) {
- tryLockFuture.cancel();
- }
- tryLockFuture = lock.asyncTryLock(lockTimeout, TimeUnit.MILLISECONDS);
- tryLockFuture.addEventListener(OrderedFutureEventListener.of(
- new FutureEventListener<LockWaiter>() {
- @Override
- public void onSuccess(LockWaiter waiter) {
- synchronized (ZKDistributedLock.this) {
- if (closed) {
- LOG.info("Skipping acquiring lock {} since it is already closed", lockPath);
- waiter.getAcquireFuture().raise(new LockingException(lockPath, "lock is already closed."));
- FutureUtils.setException(acquirePromise, newLockClosedException());
- return;
- }
- }
- tryLockFuture = null;
- lockWaiter = waiter;
- waitForAcquire(waiter, acquirePromise);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(acquirePromise, cause);
- }
- }, lockStateExecutor, lockPath));
- }
-
- void waitForAcquire(final LockWaiter waiter,
- final Promise<ZKDistributedLock> acquirePromise) {
- waiter.getAcquireFuture().addEventListener(OrderedFutureEventListener.of(
- new FutureEventListener<Boolean>() {
- @Override
- public void onSuccess(Boolean acquired) {
- LOG.info("{} acquired lock {}", waiter, lockPath);
- if (acquired) {
- FutureUtils.setValue(acquirePromise, ZKDistributedLock.this);
- } else {
- FutureUtils.setException(acquirePromise,
- new OwnershipAcquireFailedException(lockPath, waiter.getCurrentOwner()));
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(acquirePromise, cause);
- }
- }, lockStateExecutor, lockPath));
- }
-
- /**
- * NOTE: The {@link LockListener#onExpired()} is already executed in lock executor.
- */
- @Override
- public void onExpired() {
- try {
- reacquireLock(false);
- } catch (LockingException le) {
- // should not happen
- LOG.error("Locking exception on re-acquiring lock {} : ", lockPath, le);
- }
- }
-
- /**
- * Check if hold lock, if it doesn't, then re-acquire the lock.
- *
- * @throws LockingException if the lock attempt fails
- */
- public synchronized void checkOwnershipAndReacquire() throws LockingException {
- if (null == lockAcquireFuture || !lockAcquireFuture.isDefined()) {
- throw new LockingException(lockPath, "check ownership before acquiring");
- }
-
- if (haveLock()) {
- return;
- }
-
- // We may have just lost the lock because of a ZK session timeout
- // not necessarily because someone else acquired the lock.
- // In such cases just try to reacquire. If that fails, it will throw
- reacquireLock(true);
- }
-
- /**
- * Check if lock is held.
- * If not, error out and do not reacquire. Use this in cases where there are many waiters by default
- * and reacquire is unlikley to succeed.
- *
- * @throws LockingException if the lock attempt fails
- */
- public synchronized void checkOwnership() throws LockingException {
- if (null == lockAcquireFuture || !lockAcquireFuture.isDefined()) {
- throw new LockingException(lockPath, "check ownership before acquiring");
- }
- if (!haveLock()) {
- throw new LockingException(lockPath, "Lost lock ownership");
- }
- }
-
- @VisibleForTesting
- int getReacquireCount() {
- return reacquireCount.get();
- }
-
- @VisibleForTesting
- synchronized Future<ZKDistributedLock> getLockReacquireFuture() {
- return lockReacquireFuture;
- }
-
- @VisibleForTesting
- synchronized Future<ZKDistributedLock> getLockAcquireFuture() {
- return lockAcquireFuture;
- }
-
- @VisibleForTesting
- synchronized SessionLock getInternalLock() {
- return internalLock;
- }
-
- @VisibleForTesting
- LockWaiter getLockWaiter() {
- return lockWaiter;
- }
-
- synchronized boolean haveLock() {
- return !closed && internalLock != null && internalLock.isLockHeld();
- }
-
- void closeWaiter(final LockWaiter waiter,
- final Promise<Void> closePromise) {
- if (null == waiter) {
- interruptTryLock(tryLockFuture, closePromise);
- } else {
- waiter.getAcquireFuture().addEventListener(OrderedFutureEventListener.of(
- new FutureEventListener<Boolean>() {
- @Override
- public void onSuccess(Boolean value) {
- unlockInternalLock(closePromise);
- }
- @Override
- public void onFailure(Throwable cause) {
- unlockInternalLock(closePromise);
- }
- }, lockStateExecutor, lockPath));
- FutureUtils.cancel(waiter.getAcquireFuture());
- }
- }
-
- void interruptTryLock(final Future<LockWaiter> tryLockFuture,
- final Promise<Void> closePromise) {
- if (null == tryLockFuture) {
- unlockInternalLock(closePromise);
- } else {
- tryLockFuture.addEventListener(OrderedFutureEventListener.of(
- new FutureEventListener<LockWaiter>() {
- @Override
- public void onSuccess(LockWaiter waiter) {
- closeWaiter(waiter, closePromise);
- }
- @Override
- public void onFailure(Throwable cause) {
- unlockInternalLock(closePromise);
- }
- }, lockStateExecutor, lockPath));
- FutureUtils.cancel(tryLockFuture);
- }
- }
-
- synchronized void unlockInternalLock(final Promise<Void> closePromise) {
- if (internalLock == null) {
- FutureUtils.setValue(closePromise, null);
- } else {
- internalLock.asyncUnlock().ensure(new AbstractFunction0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- FutureUtils.setValue(closePromise, null);
- return BoxedUnit.UNIT;
- }
- });
- }
- }
-
- @Override
- public Future<Void> asyncClose() {
- final Promise<Void> closePromise;
- synchronized (this) {
- if (closed) {
- return closeFuture;
- }
- closed = true;
- closeFuture = closePromise = new Promise<Void>();
- }
- final Promise<Void> closeWaiterFuture = new Promise<Void>();
- closeWaiterFuture.addEventListener(OrderedFutureEventListener.of(new FutureEventListener<Void>() {
- @Override
- public void onSuccess(Void value) {
- complete();
- }
- @Override
- public void onFailure(Throwable cause) {
- complete();
- }
-
- private void complete() {
- FutureUtils.setValue(closePromise, null);
- }
- }, lockStateExecutor, lockPath));
- lockStateExecutor.submit(lockPath, new Runnable() {
- @Override
- public void run() {
- closeWaiter(lockWaiter, closeWaiterFuture);
- }
- });
- return closePromise;
- }
-
- void internalReacquireLock(final AtomicInteger numRetries,
- final long lockTimeout,
- final Promise<ZKDistributedLock> reacquirePromise) {
- lockStateExecutor.submit(lockPath, new Runnable() {
- @Override
- public void run() {
- doInternalReacquireLock(numRetries, lockTimeout, reacquirePromise);
- }
- });
- }
-
- void doInternalReacquireLock(final AtomicInteger numRetries,
- final long lockTimeout,
- final Promise<ZKDistributedLock> reacquirePromise) {
- internalTryRetries.inc();
- Promise<ZKDistributedLock> tryPromise = new Promise<ZKDistributedLock>();
- tryPromise.addEventListener(new FutureEventListener<ZKDistributedLock>() {
- @Override
- public void onSuccess(ZKDistributedLock lock) {
- FutureUtils.setValue(reacquirePromise, lock);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- if (cause instanceof OwnershipAcquireFailedException) {
- // the lock has been acquired by others
- FutureUtils.setException(reacquirePromise, cause);
- } else {
- if (numRetries.getAndDecrement() > 0 && !closed) {
- internalReacquireLock(numRetries, lockTimeout, reacquirePromise);
- } else {
- FutureUtils.setException(reacquirePromise, cause);
- }
- }
- }
- });
- doAsyncAcquireWithSemaphore(tryPromise, 0);
- }
-
- private Future<ZKDistributedLock> reacquireLock(boolean throwLockAcquireException) throws LockingException {
- final Stopwatch stopwatch = Stopwatch.createStarted();
- Promise<ZKDistributedLock> lockPromise;
- synchronized (this) {
- if (closed) {
- throw newLockClosedException();
- }
- if (null != lockReacquireException) {
- if (throwLockAcquireException) {
- throw lockReacquireException;
- } else {
- return null;
- }
- }
- if (null != lockReacquireFuture) {
- return lockReacquireFuture;
- }
- LOG.info("reacquiring lock at {}", lockPath);
- lockReacquireFuture = lockPromise = new Promise<ZKDistributedLock>();
- lockReacquireFuture.addEventListener(new FutureEventListener<ZKDistributedLock>() {
- @Override
- public void onSuccess(ZKDistributedLock lock) {
- // if re-acquire successfully, clear the state.
- synchronized (ZKDistributedLock.this) {
- lockReacquireFuture = null;
- }
- reacquireStats.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
- }
-
- @Override
- public void onFailure(Throwable cause) {
- synchronized (ZKDistributedLock.this) {
- if (cause instanceof LockingException) {
- lockReacquireException = (LockingException) cause;
- } else {
- lockReacquireException = new LockingException(lockPath,
- "Exception on re-acquiring lock", cause);
- }
- }
- reacquireStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
- }
- });
- }
- reacquireCount.incrementAndGet();
- internalReacquireLock(new AtomicInteger(Integer.MAX_VALUE), 0, lockPromise);
- return lockPromise;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLock.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLock.java
deleted file mode 100644
index dc57d55..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLock.java
+++ /dev/null
@@ -1,1363 +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.lock;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.stats.OpStatsListener;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Function0;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import com.twitter.util.TimeoutException;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.SafeRunnable;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URLDecoder;
-import java.net.URLEncoder;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * A lock under a given zookeeper session. This is a one-time lock.
- * It is not reusable: if lock failed, if zookeeper session is expired, if #unlock is called,
- * it would be transitioned to expired or closed state.
- *
- * The Locking Procedure is described as below.
- *
- * <p>
- * 0. if it is an immediate lock, it would get lock waiters first. if the lock is already held
- * by someone. it would fail immediately with {@link com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException}
- * with current owner. if there is no lock waiters, it would start locking procedure from 1.
- * 1. prepare: create a sequential znode to identify the lock.
- * 2. check lock waiters: get all lock waiters to check after prepare. if it is the first waiter,
- * claim the ownership; if it is not the first waiter, but first waiter was itself (same client id and same session id)
- * claim the ownership too; otherwise, it would set watcher on its sibling and wait it to disappared.
- * </p>
- *
- * <pre>
- * +-----------------+
- * | INIT | ------------------------------+
- * +--------+--------+ |
- * | |
- * | |
- * +--------v--------+ |
- * | PREPARING |----------------------------+ |
- * +--------+--------+ | |
- * | | |
- * | | |
- * +--------v--------+ | |
- * +-------------| PREPARED |--------------+ | |
- * | +-----^---------+-+ | | |
- * | | | | | | |
- * | | | | | | |
- * | | | | | | |
- * +------V-----------+ | | | +--------v----------+ | |
- * | WAITING |-------+ | | | CLAIMED | | |
- * +------+-----+-----+ | | +--+----------+-----+ | |
- * | | | | | | | |
- * | | | | | | | |
- * | | | | | | | |
- * | | | +-v----------v----+ | | |
- * | +-------------------->| EXPIRED | | | |
- * | | +--+--------------+ | | |
- * | | | | | |
- * | | | | | |
- * | +--------V-------V-+ | | |
- * +------------>| CLOSING |<---------------+----------+--+
- * +------------------+
- * |
- * |
- * |
- * +--------V---------+
- * | CLOSED |
- * +------------------+
- * </pre>
- *
- * <h3>Metrics</h3>
- * <ul>
- * <li>tryAcquire: opstats. latency spent on try locking operations. it includes timeouts.
- * <li>tryTimeouts: counter. the number of timeouts on try locking operations
- * <li>unlock: opstats. latency spent on unlock operations.
- * </ul>
- */
-class ZKSessionLock implements SessionLock {
-
- static final Logger LOG = LoggerFactory.getLogger(ZKSessionLock.class);
-
- private static final String LOCK_PATH_PREFIX = "/member_";
- private static final String LOCK_PART_SEP = "_";
-
- public static String getLockPathPrefixV1(String lockPath) {
- // member_
- return lockPath + LOCK_PATH_PREFIX;
- }
-
- public static String getLockPathPrefixV2(String lockPath, String clientId) throws UnsupportedEncodingException {
- // member_<clientid>_
- return lockPath + LOCK_PATH_PREFIX + URLEncoder.encode(clientId, UTF_8.name()) + LOCK_PART_SEP;
- }
-
- public static String getLockPathPrefixV3(String lockPath, String clientId, long sessionOwner) throws UnsupportedEncodingException {
- // member_<clientid>_s<owner_session>_
- StringBuilder sb = new StringBuilder();
- sb.append(lockPath).append(LOCK_PATH_PREFIX).append(URLEncoder.encode(clientId, UTF_8.name())).append(LOCK_PART_SEP)
- .append("s").append(String.format("%10d", sessionOwner)).append(LOCK_PART_SEP);
- return sb.toString();
- }
-
- public static byte[] serializeClientId(String clientId) {
- return clientId.getBytes(UTF_8);
- }
-
- public static String deserializeClientId(byte[] data) {
- return new String(data, UTF_8);
- }
-
- public static String getLockIdFromPath(String path) {
- // We only care about our actual id since we want to compare ourselves to siblings.
- if (path.contains("/")) {
- return path.substring(path.lastIndexOf("/") + 1);
- } else {
- return path;
- }
- }
-
- static final Comparator<String> MEMBER_COMPARATOR = new Comparator<String>() {
- public int compare(String o1, String o2) {
- int l1 = parseMemberID(o1);
- int l2 = parseMemberID(o2);
- return l1 - l2;
- }
- };
-
- static enum State {
- INIT, // initialized state
- PREPARING, // preparing to lock, but no lock node created
- PREPARED, // lock node created
- CLAIMED, // claim lock ownership
- WAITING, // waiting for the ownership
- EXPIRED, // lock is expired
- CLOSING, // lock is being closed
- CLOSED, // lock is closed
- }
-
- /**
- * Convenience class for state management. Provide debuggability features by tracing unxpected state
- * transitions.
- */
- static class StateManagement {
-
- static final Logger LOG = LoggerFactory.getLogger(StateManagement.class);
-
- private volatile State state;
-
- StateManagement() {
- this.state = State.INIT;
- }
-
- public void transition(State toState) {
- if (!validTransition(toState)) {
- LOG.error("Invalid state transition from {} to {} ",
- new Object[] { this.state, toState, getStack() });
- }
- this.state = toState;
- }
-
- private boolean validTransition(State toState) {
- switch (toState) {
- case INIT:
- return false;
- case PREPARING:
- return inState(State.INIT);
- case PREPARED:
- return inState(State.PREPARING) || inState(State.WAITING);
- case CLAIMED:
- return inState(State.PREPARED);
- case WAITING:
- return inState(State.PREPARED);
- case EXPIRED:
- return isTryingOrClaimed();
- case CLOSING:
- return !inState(State.CLOSED);
- case CLOSED:
- return inState(State.CLOSING) || inState(State.CLOSED);
- default:
- return false;
- }
- }
-
- private State getState() {
- return state;
- }
-
- private boolean isTryingOrClaimed() {
- return inState(State.PREPARING) || inState(State.PREPARED) ||
- inState(State.WAITING) || inState(State.CLAIMED);
- }
-
- public boolean isExpiredOrClosing() {
- return inState(State.CLOSED) || inState(State.EXPIRED) || inState(State.CLOSING);
- }
-
- public boolean isExpiredOrClosed() {
- return inState(State.CLOSED) || inState(State.EXPIRED);
- }
-
- public boolean isClosed() {
- return inState(State.CLOSED);
- }
-
- private boolean inState(final State state) {
- return state == this.state;
- }
-
- private Exception getStack() {
- return new Exception();
- }
- }
-
- private final ZooKeeperClient zkClient;
- private final ZooKeeper zk;
- private final String lockPath;
- // Identify a unique lock
- private final Pair<String, Long> lockId;
- private StateManagement lockState;
- private final DistributedLockContext lockContext;
-
- private final Promise<Boolean> acquireFuture;
- private String currentId;
- private String currentNode;
- private String watchedNode;
- private LockWatcher watcher;
- private final AtomicInteger epoch = new AtomicInteger(0);
- private final OrderedScheduler lockStateExecutor;
- private LockListener lockListener = null;
- private final long lockOpTimeout;
-
- private final OpStatsLogger tryStats;
- private final Counter tryTimeouts;
- private final OpStatsLogger unlockStats;
-
- ZKSessionLock(ZooKeeperClient zkClient,
- String lockPath,
- String clientId,
- OrderedScheduler lockStateExecutor)
- throws IOException {
- this(zkClient,
- lockPath,
- clientId,
- lockStateExecutor,
- DistributedLogConstants.LOCK_OP_TIMEOUT_DEFAULT * 1000, NullStatsLogger.INSTANCE,
- new DistributedLockContext());
- }
-
- /**
- * Creates a distributed lock using the given {@code zkClient} to coordinate locking.
- *
- * @param zkClient The ZooKeeper client to use.
- * @param lockPath The path used to manage the lock under.
- * @param clientId client id use for lock.
- * @param lockStateExecutor executor to execute all lock state changes.
- * @param lockOpTimeout timeout of lock operations
- * @param statsLogger stats logger
- */
- public ZKSessionLock(ZooKeeperClient zkClient,
- String lockPath,
- String clientId,
- OrderedScheduler lockStateExecutor,
- long lockOpTimeout,
- StatsLogger statsLogger,
- DistributedLockContext lockContext)
- throws IOException {
- this.zkClient = zkClient;
- try {
- this.zk = zkClient.get();
- } catch (ZooKeeperClient.ZooKeeperConnectionException zce) {
- throw new ZKException("Failed to get zookeeper client for lock " + lockPath,
- KeeperException.Code.CONNECTIONLOSS);
- } catch (InterruptedException e) {
- throw new DLInterruptedException("Interrupted on getting zookeeper client for lock " + lockPath, e);
- }
- this.lockPath = lockPath;
- this.lockId = Pair.of(clientId, this.zk.getSessionId());
- this.lockContext = lockContext;
- this.lockStateExecutor = lockStateExecutor;
- this.lockState = new StateManagement();
- this.lockOpTimeout = lockOpTimeout;
-
- this.tryStats = statsLogger.getOpStatsLogger("tryAcquire");
- this.tryTimeouts = statsLogger.getCounter("tryTimeouts");
- this.unlockStats = statsLogger.getOpStatsLogger("unlock");
-
- // Attach interrupt handler to acquire future so clients can abort the future.
- this.acquireFuture = new Promise<Boolean>(new com.twitter.util.Function<Throwable, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Throwable t) {
- // This will set the lock state to closed, and begin to cleanup the zk lock node.
- // We have to be careful not to block here since doing so blocks the ordered lock
- // state executor which can cause deadlocks depending on how futures are chained.
- ZKSessionLock.this.asyncUnlock(t);
- // Note re. logging and exceptions: errors are already logged by unlockAsync.
- return BoxedUnit.UNIT;
- }
- });
- }
-
- @Override
- public ZKSessionLock setLockListener(LockListener lockListener) {
- this.lockListener = lockListener;
- return this;
- }
-
- String getLockPath() {
- return this.lockPath;
- }
-
- @VisibleForTesting
- AtomicInteger getEpoch() {
- return epoch;
- }
-
- @VisibleForTesting
- State getLockState() {
- return lockState.getState();
- }
-
- @VisibleForTesting
- Pair<String, Long> getLockId() {
- return lockId;
- }
-
- public boolean isLockExpired() {
- return lockState.isExpiredOrClosing();
- }
-
- @Override
- public boolean isLockHeld() {
- return lockState.inState(State.CLAIMED);
- }
-
- /**
- * Execute a lock action of a given <i>lockEpoch</i> in ordered safe way.
- *
- * @param lockEpoch
- * lock epoch
- * @param func
- * function to execute a lock action
- */
- protected void executeLockAction(final int lockEpoch, final LockAction func) {
- lockStateExecutor.submit(lockPath, new SafeRunnable() {
- @Override
- public void safeRun() {
- if (ZKSessionLock.this.epoch.get() == lockEpoch) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("{} executing lock action '{}' under epoch {} for lock {}",
- new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
- }
- func.execute();
- if (LOG.isTraceEnabled()) {
- LOG.trace("{} executed lock action '{}' under epoch {} for lock {}",
- new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
- }
- } else {
- if (LOG.isTraceEnabled()) {
- LOG.trace("{} skipped executing lock action '{}' for lock {}, since epoch is changed from {} to {}.",
- new Object[]{lockId, func.getActionName(), lockPath, lockEpoch, ZKSessionLock.this.epoch.get()});
- }
- }
- }
- });
- }
-
- /**
- * Execute a lock action of a given <i>lockEpoch</i> in ordered safe way. If the lock action couln't be
- * executed due to epoch changed, fail the given <i>promise</i> with
- * {@link EpochChangedException}
- *
- * @param lockEpoch
- * lock epoch
- * @param func
- * function to execute a lock action
- * @param promise
- * promise
- */
- protected <T> void executeLockAction(final int lockEpoch, final LockAction func, final Promise<T> promise) {
- lockStateExecutor.submit(lockPath, new SafeRunnable() {
- @Override
- public void safeRun() {
- int currentEpoch = ZKSessionLock.this.epoch.get();
- if (currentEpoch == lockEpoch) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("{} executed lock action '{}' under epoch {} for lock {}",
- new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
- }
- func.execute();
- if (LOG.isTraceEnabled()) {
- LOG.trace("{} executed lock action '{}' under epoch {} for lock {}",
- new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
- }
- } else {
- if (LOG.isTraceEnabled()) {
- LOG.trace("{} skipped executing lock action '{}' for lock {}, since epoch is changed from {} to {}.",
- new Object[]{lockId, func.getActionName(), lockPath, lockEpoch, currentEpoch});
- }
- promise.setException(new EpochChangedException(lockPath, lockEpoch, currentEpoch));
- }
- }
- });
- }
-
- /**
- * Parse member id generated by zookeeper from given <i>nodeName</i>
- *
- * @param nodeName
- * lock node name
- * @return member id generated by zookeeper
- */
- static int parseMemberID(String nodeName) {
- int id = -1;
- String[] parts = nodeName.split("_");
- if (parts.length > 0) {
- try {
- id = Integer.parseInt(parts[parts.length - 1]);
- } catch (NumberFormatException nfe) {
- // make it to be MAX_VALUE, so the bad znode will never acquire the lock
- id = Integer.MAX_VALUE;
- }
- }
- return id;
- }
-
- static boolean areLockWaitersInSameSession(String node1, String node2) {
- String[] parts1 = node1.split("_");
- String[] parts2 = node2.split("_");
- if (parts1.length != 4 || parts2.length != 4) {
- return node1.equals(node2);
- }
- if (!parts1[2].startsWith("s") || !parts2[2].startsWith("s")) {
- return node1.equals(node2);
- }
- long sessionOwner1 = Long.parseLong(parts1[2].substring(1));
- long sessionOwner2 = Long.parseLong(parts2[2].substring(1));
- if (sessionOwner1 != sessionOwner2) {
- return false;
- }
- String clientId1, clientId2;
- try {
- clientId1 = URLDecoder.decode(parts1[1], UTF_8.name());
- clientId2 = URLDecoder.decode(parts2[1], UTF_8.name());
- return clientId1.equals(clientId2);
- } catch (UnsupportedEncodingException e) {
- // if failed to parse client id, we have to get client id by zookeeper#getData.
- return node1.equals(node2);
- }
- }
-
- /**
- * Get client id and its ephemeral owner.
- *
- * @param zkClient
- * zookeeper client
- * @param lockPath
- * lock path
- * @param nodeName
- * node name
- * @return client id and its ephemeral owner.
- */
- static Future<Pair<String, Long>> asyncParseClientID(ZooKeeper zkClient, String lockPath, String nodeName) {
- String[] parts = nodeName.split("_");
- // member_<clientid>_s<owner_session>_
- if (4 == parts.length && parts[2].startsWith("s")) {
- long sessionOwner = Long.parseLong(parts[2].substring(1));
- String clientId;
- try {
- clientId = URLDecoder.decode(parts[1], UTF_8.name());
- return Future.value(Pair.of(clientId, sessionOwner));
- } catch (UnsupportedEncodingException e) {
- // if failed to parse client id, we have to get client id by zookeeper#getData.
- }
- }
- final Promise<Pair<String, Long>> promise = new Promise<Pair<String, Long>>();
- zkClient.getData(lockPath + "/" + nodeName, false, new AsyncCallback.DataCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
- if (KeeperException.Code.OK.intValue() != rc) {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- } else {
- promise.setValue(Pair.of(deserializeClientId(data), stat.getEphemeralOwner()));
- }
- }
- }, null);
- return promise;
- }
-
- @Override
- public Future<LockWaiter> asyncTryLock(final long timeout, final TimeUnit unit) {
- final Promise<String> result = new Promise<String>();
- final boolean wait = DistributedLogConstants.LOCK_IMMEDIATE != timeout;
- if (wait) {
- asyncTryLock(wait, result);
- } else {
- // try to check locks first
- zk.getChildren(lockPath, null, new AsyncCallback.Children2Callback() {
- @Override
- public void processResult(final int rc, String path, Object ctx,
- final List<String> children, Stat stat) {
- lockStateExecutor.submit(lockPath, new SafeRunnable() {
- @Override
- public void safeRun() {
- if (!lockState.inState(State.INIT)) {
- result.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
- return;
- }
- if (KeeperException.Code.OK.intValue() != rc) {
- result.setException(KeeperException.create(KeeperException.Code.get(rc)));
- return;
- }
-
- FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockTryAcquire);
-
- Collections.sort(children, MEMBER_COMPARATOR);
- if (children.size() > 0) {
- asyncParseClientID(zk, lockPath, children.get(0)).addEventListener(
- new FutureEventListener<Pair<String, Long>>() {
- @Override
- public void onSuccess(Pair<String, Long> owner) {
- if (!checkOrClaimLockOwner(owner, result)) {
- acquireFuture.updateIfEmpty(new Return<Boolean>(false));
- }
- }
-
- @Override
- public void onFailure(final Throwable cause) {
- lockStateExecutor.submit(lockPath, new SafeRunnable() {
- @Override
- public void safeRun() {
- result.setException(cause);
- }
- });
- }
- });
- } else {
- asyncTryLock(wait, result);
- }
- }
- });
- }
- }, null);
- }
-
- final Promise<Boolean> waiterAcquireFuture = new Promise<Boolean>(new com.twitter.util.Function<Throwable, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Throwable t) {
- acquireFuture.raise(t);
- return BoxedUnit.UNIT;
- }
- });
- return result.map(new AbstractFunction1<String, LockWaiter>() {
- @Override
- public LockWaiter apply(final String currentOwner) {
- final Exception acquireException = new OwnershipAcquireFailedException(lockPath, currentOwner);
- FutureUtils.within(
- acquireFuture,
- timeout,
- unit,
- acquireException,
- lockStateExecutor,
- lockPath
- ).addEventListener(new FutureEventListener<Boolean>() {
-
- @Override
- public void onSuccess(Boolean acquired) {
- completeOrFail(acquireException);
- }
-
- @Override
- public void onFailure(final Throwable acquireCause) {
- completeOrFail(acquireException);
- }
-
- private void completeOrFail(final Throwable acquireCause) {
- if (isLockHeld()) {
- waiterAcquireFuture.setValue(true);
- } else {
- asyncUnlock().addEventListener(new FutureEventListener<BoxedUnit>() {
- @Override
- public void onSuccess(BoxedUnit value) {
- waiterAcquireFuture.setException(acquireCause);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- waiterAcquireFuture.setException(acquireCause);
- }
- });
- }
- }
- });;
- return new LockWaiter(
- lockId.getLeft(),
- currentOwner,
- waiterAcquireFuture);
- }
- });
- }
-
- private boolean checkOrClaimLockOwner(final Pair<String, Long> currentOwner,
- final Promise<String> result) {
- if (lockId.compareTo(currentOwner) != 0 && !lockContext.hasLockId(currentOwner)) {
- lockStateExecutor.submit(lockPath, new SafeRunnable() {
- @Override
- public void safeRun() {
- result.setValue(currentOwner.getLeft());
- }
- });
- return false;
- }
- // current owner is itself
- final int curEpoch = epoch.incrementAndGet();
- executeLockAction(curEpoch, new LockAction() {
- @Override
- public void execute() {
- if (!lockState.inState(State.INIT)) {
- result.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
- return;
- }
- asyncTryLock(false, result);
- }
- @Override
- public String getActionName() {
- return "claimOwnership(owner=" + currentOwner + ")";
- }
- }, result);
- return true;
- }
-
- /**
- * Try lock. If it failed, it would cleanup its attempt.
- *
- * @param wait
- * whether to wait for ownership.
- * @param result
- * promise to satisfy with current lock owner
- */
- private void asyncTryLock(boolean wait, final Promise<String> result) {
- final Promise<String> lockResult = new Promise<String>();
- lockResult.addEventListener(new FutureEventListener<String>() {
- @Override
- public void onSuccess(String currentOwner) {
- result.setValue(currentOwner);
- }
-
- @Override
- public void onFailure(final Throwable lockCause) {
- // If tryLock failed due to state changed, we don't need to cleanup
- if (lockCause instanceof LockStateChangedException) {
- LOG.info("skipping cleanup for {} at {} after encountering lock " +
- "state change exception : ", new Object[] { lockId, lockPath, lockCause });
- result.setException(lockCause);
- return;
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} is cleaning up its lock state for {} due to : ",
- new Object[] { lockId, lockPath, lockCause });
- }
-
- // If we encountered any exception we should cleanup
- Future<BoxedUnit> unlockResult = asyncUnlock();
- unlockResult.addEventListener(new FutureEventListener<BoxedUnit>() {
- @Override
- public void onSuccess(BoxedUnit value) {
- result.setException(lockCause);
- }
- @Override
- public void onFailure(Throwable cause) {
- result.setException(lockCause);
- }
- });
- }
- });
- asyncTryLockWithoutCleanup(wait, lockResult);
- }
-
- /**
- * Try lock. If wait is true, it would wait and watch sibling to acquire lock when
- * the sibling is dead. <i>acquireFuture</i> will be notified either it locked successfully
- * or the lock failed. The promise will only satisfy with current lock owner.
- *
- * NOTE: the <i>promise</i> is only satisfied on <i>lockStateExecutor</i>, so any
- * transformations attached on promise will be executed in order.
- *
- * @param wait
- * whether to wait for ownership.
- * @param promise
- * promise to satisfy with current lock owner.
- */
- private void asyncTryLockWithoutCleanup(final boolean wait, final Promise<String> promise) {
- executeLockAction(epoch.get(), new LockAction() {
- @Override
- public void execute() {
- if (!lockState.inState(State.INIT)) {
- promise.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
- return;
- }
- lockState.transition(State.PREPARING);
-
- final int curEpoch = epoch.incrementAndGet();
- watcher = new LockWatcher(curEpoch);
- // register watcher for session expires
- zkClient.register(watcher);
- // Encode both client id and session in the lock node
- String myPath;
- try {
- // member_<clientid>_s<owner_session>_
- myPath = getLockPathPrefixV3(lockPath, lockId.getLeft(), lockId.getRight());
- } catch (UnsupportedEncodingException uee) {
- myPath = getLockPathPrefixV1(lockPath);
- }
- zk.create(myPath, serializeClientId(lockId.getLeft()), zkClient.getDefaultACL(), CreateMode.EPHEMERAL_SEQUENTIAL,
- new AsyncCallback.StringCallback() {
- @Override
- public void processResult(final int rc, String path, Object ctx, final String name) {
- executeLockAction(curEpoch, new LockAction() {
- @Override
- public void execute() {
- if (KeeperException.Code.OK.intValue() != rc) {
- KeeperException ke = KeeperException.create(KeeperException.Code.get(rc));
- promise.setException(ke);
- return;
- }
-
- if (FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition)) {
- lockState.transition(State.CLOSING);
- lockState.transition(State.CLOSED);
- }
-
- if (null != currentNode) {
- LOG.error("Current node for {} overwritten current = {} new = {}",
- new Object[] { lockPath, lockId, getLockIdFromPath(currentNode) });
- }
-
- currentNode = name;
- currentId = getLockIdFromPath(currentNode);
- LOG.trace("{} received member id for lock {}", lockId, currentId);
-
- if (lockState.isExpiredOrClosing()) {
- // Delete node attempt may have come after PREPARING but before create node, in which case
- // we'd be left with a dangling node unless we clean up.
- Promise<BoxedUnit> deletePromise = new Promise<BoxedUnit>();
- deleteLockNode(deletePromise);
- deletePromise.ensure(new Function0<BoxedUnit>() {
- public BoxedUnit apply() {
- promise.setException(new LockClosedException(lockPath, lockId, lockState.getState()));
- return BoxedUnit.UNIT;
- }
- });
- return;
- }
-
- lockState.transition(State.PREPARED);
- checkLockOwnerAndWaitIfPossible(watcher, wait, promise);
- }
-
- @Override
- public String getActionName() {
- return "postPrepare(wait=" + wait + ")";
- }
- });
- }
- }, null);
- }
- @Override
- public String getActionName() {
- return "prepare(wait=" + wait + ")";
- }
- }, promise);
- }
-
- @Override
- public void tryLock(long timeout, TimeUnit unit) throws LockingException {
- final Stopwatch stopwatch = Stopwatch.createStarted();
- Future<LockWaiter> tryFuture = asyncTryLock(timeout, unit);
- LockWaiter waiter = waitForTry(stopwatch, tryFuture);
- boolean acquired = waiter.waitForAcquireQuietly();
- if (!acquired) {
- throw new OwnershipAcquireFailedException(lockPath, waiter.getCurrentOwner());
- }
- }
-
- synchronized LockWaiter waitForTry(Stopwatch stopwatch, Future<LockWaiter> tryFuture)
- throws LockingException {
- boolean success = false;
- boolean stateChanged = false;
- LockWaiter waiter;
- try {
- waiter = Await.result(tryFuture, Duration.fromMilliseconds(lockOpTimeout));
- success = true;
- } catch (LockStateChangedException ex) {
- stateChanged = true;
- throw ex;
- } catch (LockingException ex) {
- throw ex;
- } catch (TimeoutException toe) {
- tryTimeouts.inc();
- throw new LockingException(lockPath, "Timeout during try phase of lock acquire", toe);
- } catch (Exception ex) {
- String message = getLockId() + " failed to lock " + lockPath;
- throw new LockingException(lockPath, message, ex);
- } finally {
- if (success) {
- tryStats.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
- } else {
- tryStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
- }
- // This can only happen for a Throwable thats not an
- // Exception, i.e. an Error
- if (!success && !stateChanged) {
- unlock();
- }
- }
- return waiter;
- }
-
- @Override
- public Future<BoxedUnit> asyncUnlock() {
- return asyncUnlock(new LockClosedException(lockPath, lockId, lockState.getState()));
- }
-
- Future<BoxedUnit> asyncUnlock(final Throwable cause) {
- final Promise<BoxedUnit> promise = new Promise<BoxedUnit>();
-
- // Use lock executor here rather than lock action, because we want this opertaion to be applied
- // whether the epoch has changed or not. The member node is EPHEMERAL_SEQUENTIAL so there's no
- // risk of an ABA problem where we delete and recreate a node and then delete it again here.
- lockStateExecutor.submit(lockPath, new SafeRunnable() {
- @Override
- public void safeRun() {
- acquireFuture.updateIfEmpty(new Throw<Boolean>(cause));
- unlockInternal(promise);
- promise.addEventListener(new OpStatsListener<BoxedUnit>(unlockStats));
- }
- });
-
- return promise;
- }
-
- @Override
- public void unlock() {
- Future<BoxedUnit> unlockResult = asyncUnlock();
- try {
- Await.result(unlockResult, Duration.fromMilliseconds(lockOpTimeout));
- } catch (TimeoutException toe) {
- // This shouldn't happen unless we lose a watch, and may result in a leaked lock.
- LOG.error("Timeout unlocking {} owned by {} : ", new Object[] { lockPath, lockId, toe });
- } catch (Exception e) {
- LOG.warn("{} failed to unlock {} : ", new Object[] { lockId, lockPath, e });
- }
- }
-
- // Lock State Changes (all state changes should be executed under a LockAction)
-
- private void claimOwnership(int lockEpoch) {
- lockState.transition(State.CLAIMED);
- // clear previous lock ids
- lockContext.clearLockIds();
- // add current lock id
- lockContext.addLockId(lockId);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Notify lock waiters on {} at {} : watcher epoch {}, lock epoch {}",
- new Object[] { lockPath, System.currentTimeMillis(),
- lockEpoch, ZKSessionLock.this.epoch.get() });
- }
- acquireFuture.updateIfEmpty(new Return<Boolean>(true));
- }
-
- /**
- * NOTE: unlockInternal should only after try lock.
- */
- private void unlockInternal(final Promise<BoxedUnit> promise) {
-
- // already closed or expired, nothing to cleanup
- this.epoch.incrementAndGet();
- if (null != watcher) {
- this.zkClient.unregister(watcher);
- }
-
- if (lockState.inState(State.CLOSED)) {
- promise.setValue(BoxedUnit.UNIT);
- return;
- }
-
- LOG.info("Lock {} for {} is closed from state {}.",
- new Object[] { lockId, lockPath, lockState.getState() });
-
- final boolean skipCleanup = lockState.inState(State.INIT) || lockState.inState(State.EXPIRED);
-
- lockState.transition(State.CLOSING);
-
- if (skipCleanup) {
- // Nothing to cleanup if INIT (never tried) or EXPIRED (ephemeral node
- // auto-removed)
- lockState.transition(State.CLOSED);
- promise.setValue(BoxedUnit.UNIT);
- return;
- }
-
- // In any other state, we should clean up the member node
- Promise<BoxedUnit> deletePromise = new Promise<BoxedUnit>();
- deleteLockNode(deletePromise);
-
- // Set the state to closed after we've cleaned up
- deletePromise.addEventListener(new FutureEventListener<BoxedUnit>() {
- @Override
- public void onSuccess(BoxedUnit complete) {
- lockStateExecutor.submit(lockPath, new SafeRunnable() {
- @Override
- public void safeRun() {
- lockState.transition(State.CLOSED);
- promise.setValue(BoxedUnit.UNIT);
- }
- });
- }
- @Override
- public void onFailure(Throwable cause) {
- // Delete failure is quite serious (causes lock leak) and should be
- // handled better
- LOG.error("lock node delete failed {} {}", lockId, lockPath);
- promise.setValue(BoxedUnit.UNIT);
- }
- });
- }
-
- private void deleteLockNode(final Promise<BoxedUnit> promise) {
- if (null == currentNode) {
- promise.setValue(BoxedUnit.UNIT);
- return;
- }
-
- zk.delete(currentNode, -1, new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(final int rc, final String path, Object ctx) {
- lockStateExecutor.submit(lockPath, new SafeRunnable() {
- @Override
- public void safeRun() {
- if (KeeperException.Code.OK.intValue() == rc) {
- LOG.info("Deleted lock node {} for {} successfully.", path, lockId);
- } else if (KeeperException.Code.NONODE.intValue() == rc ||
- KeeperException.Code.SESSIONEXPIRED.intValue() == rc) {
- LOG.info("Delete node failed. Node already gone for node {} id {}, rc = {}",
- new Object[] { path, lockId, KeeperException.Code.get(rc) });
- } else {
- LOG.error("Failed on deleting lock node {} for {} : {}",
- new Object[] { path, lockId, KeeperException.Code.get(rc) });
- }
-
- FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockUnlockCleanup);
- promise.setValue(BoxedUnit.UNIT);
- }
- });
- }
- }, null);
- }
-
- /**
- * Handle session expired for lock watcher at epoch <i>lockEpoch</i>.
- *
- * @param lockEpoch
- * lock epoch
- */
- private void handleSessionExpired(final int lockEpoch) {
- executeLockAction(lockEpoch, new LockAction() {
- @Override
- public void execute() {
- if (lockState.inState(State.CLOSED) || lockState.inState(State.CLOSING)) {
- // Already fully closed, no need to process expire.
- return;
- }
-
- boolean shouldNotifyLockListener = lockState.inState(State.CLAIMED);
-
- lockState.transition(State.EXPIRED);
-
- // remove the watcher
- if (null != watcher) {
- zkClient.unregister(watcher);
- }
-
- // increment epoch to avoid any ongoing locking action
- ZKSessionLock.this.epoch.incrementAndGet();
-
- // if session expired, just notify the waiter. as the lock acquire doesn't succeed.
- // we don't even need to clean up the lock as the znode will disappear after session expired
- acquireFuture.updateIfEmpty(new Throw<Boolean>(
- new LockSessionExpiredException(lockPath, lockId, lockState.getState())));
-
- // session expired, ephemeral node is gone.
- currentNode = null;
- currentId = null;
-
- if (shouldNotifyLockListener) {
- // if session expired after claimed, we need to notify the caller to re-lock
- if (null != lockListener) {
- lockListener.onExpired();
- }
- }
- }
-
- @Override
- public String getActionName() {
- return "handleSessionExpired(epoch=" + lockEpoch + ")";
- }
- });
- }
-
- private void handleNodeDelete(int lockEpoch, final WatchedEvent event) {
- executeLockAction(lockEpoch, new LockAction() {
- @Override
- public void execute() {
- // The lock is either expired or closed
- if (!lockState.inState(State.WAITING)) {
- LOG.info("{} ignore watched node {} deleted event, since lock state has moved to {}.",
- new Object[] { lockId, event.getPath(), lockState.getState() });
- return;
- }
- lockState.transition(State.PREPARED);
-
- // we don't need to wait and check the result, since:
- // 1) if it claimed the ownership, it would notify the waiters when claimed ownerships
- // 2) if it failed, it would also notify the waiters, the waiters would cleanup the state.
- checkLockOwnerAndWaitIfPossible(watcher, true);
- }
-
- @Override
- public String getActionName() {
- return "handleNodeDelete(path=" + event.getPath() + ")";
- }
- });
- }
-
- private Future<String> checkLockOwnerAndWaitIfPossible(final LockWatcher lockWatcher,
- final boolean wait) {
- final Promise<String> promise = new Promise<String>();
- checkLockOwnerAndWaitIfPossible(lockWatcher, wait, promise);
- return promise;
- }
-
- /**
- * Check Lock Owner Phase 1 : Get all lock waiters.
- *
- * @param lockWatcher
- * lock watcher.
- * @param wait
- * whether to wait for ownership.
- * @param promise
- * promise to satisfy with current lock owner
- */
- private void checkLockOwnerAndWaitIfPossible(final LockWatcher lockWatcher,
- final boolean wait,
- final Promise<String> promise) {
- zk.getChildren(lockPath, false, new AsyncCallback.Children2Callback() {
- @Override
- public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
- processLockWaiters(lockWatcher, wait, rc, children, promise);
- }
- }, null);
- }
-
- /**
- * Check Lock Owner Phase 2 : check all lock waiters to get current owner and wait for ownership if necessary.
- *
- * @param lockWatcher
- * lock watcher.
- * @param wait
- * whether to wait for ownership.
- * @param getChildrenRc
- * result of getting all lock waiters
- * @param children
- * current lock waiters.
- * @param promise
- * promise to satisfy with current lock owner.
- */
- private void processLockWaiters(final LockWatcher lockWatcher,
- final boolean wait,
- final int getChildrenRc,
- final List<String> children,
- final Promise<String> promise) {
- executeLockAction(lockWatcher.epoch, new LockAction() {
- @Override
- public void execute() {
- if (!lockState.inState(State.PREPARED)) { // e.g. lock closed or session expired after prepared
- promise.setException(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
- return;
- }
-
- if (KeeperException.Code.OK.intValue() != getChildrenRc) {
- promise.setException(KeeperException.create(KeeperException.Code.get(getChildrenRc)));
- return;
- }
- if (children.isEmpty()) {
- LOG.error("Error, member list is empty for lock {}.", lockPath);
- promise.setException(new UnexpectedException("Empty member list for lock " + lockPath));
- return;
- }
-
- // sort the children
- Collections.sort(children, MEMBER_COMPARATOR);
- final String cid = currentId;
- final int memberIndex = children.indexOf(cid);
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} is the number {} member in the list.", cid, memberIndex);
- }
- // If we hold the lock
- if (memberIndex == 0) {
- LOG.info("{} acquired the lock {}.", cid, lockPath);
- claimOwnership(lockWatcher.epoch);
- promise.setValue(cid);
- } else if (memberIndex > 0) { // we are in the member list but we didn't hold the lock
- // get ownership of current owner
- asyncParseClientID(zk, lockPath, children.get(0)).addEventListener(new FutureEventListener<Pair<String, Long>>() {
- @Override
- public void onSuccess(Pair<String, Long> currentOwner) {
- watchLockOwner(lockWatcher, wait,
- cid, children.get(memberIndex - 1), children.get(0), currentOwner, promise);
- }
- @Override
- public void onFailure(final Throwable cause) {
- // ensure promise is satisfied in lock thread
- executeLockAction(lockWatcher.epoch, new LockAction() {
- @Override
- public void execute() {
- promise.setException(cause);
- }
-
- @Override
- public String getActionName() {
- return "handleFailureOnParseClientID(lockPath=" + lockPath + ")";
- }
- }, promise);
- }
- });
- } else {
- LOG.error("Member {} doesn't exist in the members list {} for lock {}.",
- new Object[]{ cid, children, lockPath});
- promise.setException(
- new UnexpectedException("Member " + cid + " doesn't exist in member list " +
- children + " for lock " + lockPath));
- }
- }
-
- @Override
- public String getActionName() {
- return "processLockWaiters(rc=" + getChildrenRc + ", waiters=" + children + ")";
- }
- }, promise);
- }
-
- /**
- * Check Lock Owner Phase 3: watch sibling node for lock ownership.
- *
- * @param lockWatcher
- * lock watcher.
- * @param wait
- * whether to wait for ownership.
- * @param myNode
- * my lock node.
- * @param siblingNode
- * my sibling lock node.
- * @param ownerNode
- * owner lock node.
- * @param currentOwner
- * current owner info.
- * @param promise
- * promise to satisfy with current lock owner.
- */
- private void watchLockOwner(final LockWatcher lockWatcher,
- final boolean wait,
- final String myNode,
- final String siblingNode,
- final String ownerNode,
- final Pair<String, Long> currentOwner,
- final Promise<String> promise) {
- executeLockAction(lockWatcher.epoch, new LockAction() {
- @Override
- public void execute() {
- boolean shouldWatch;
- final boolean shouldClaimOwnership;
- if (lockContext.hasLockId(currentOwner) && siblingNode.equals(ownerNode)) {
- // if the current owner is the znode left from previous session
- // we should watch it and claim ownership
- shouldWatch = true;
- shouldClaimOwnership = true;
- LOG.info("LockWatcher {} for {} found its previous session {} held lock, watch it to claim ownership.",
- new Object[] { myNode, lockPath, currentOwner });
- } else if (lockId.compareTo(currentOwner) == 0 && areLockWaitersInSameSession(siblingNode, ownerNode)) {
- // I found that my sibling is the current owner with same lock id (client id & session id)
- // It must be left by any race condition from same zookeeper client
- shouldWatch = true;
- shouldClaimOwnership = true;
- LOG.info("LockWatcher {} for {} found itself {} already held lock at sibling node {}, watch it to claim ownership.",
- new Object[]{myNode, lockPath, lockId, siblingNode});
- } else {
- shouldWatch = wait;
- if (wait) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Current LockWatcher for {} with ephemeral node {}, is waiting for {} to release lock at {}.",
- new Object[]{lockPath, myNode, siblingNode, System.currentTimeMillis()});
- }
- }
- shouldClaimOwnership = false;
- }
-
- // watch sibling for lock ownership
- if (shouldWatch) {
- watchedNode = String.format("%s/%s", lockPath, siblingNode);
- zk.exists(watchedNode, lockWatcher, new AsyncCallback.StatCallback() {
- @Override
- public void processResult(final int rc, String path, Object ctx, final Stat stat) {
- executeLockAction(lockWatcher.epoch, new LockAction() {
- @Override
- public void execute() {
- if (!lockState.inState(State.PREPARED)) {
- promise.setException(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
- return;
- }
-
- if (KeeperException.Code.OK.intValue() == rc) {
- if (shouldClaimOwnership) {
- // watch owner successfully
- LOG.info("LockWatcher {} claimed ownership for {} after set watcher on {}.",
- new Object[]{ myNode, lockPath, ownerNode });
- claimOwnership(lockWatcher.epoch);
- promise.setValue(currentOwner.getLeft());
- } else {
- // watch sibling successfully
- lockState.transition(State.WAITING);
- promise.setValue(currentOwner.getLeft());
- }
- } else if (KeeperException.Code.NONODE.intValue() == rc) {
- // sibling just disappeared, it might be the chance to claim ownership
- checkLockOwnerAndWaitIfPossible(lockWatcher, wait, promise);
- } else {
- promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
- }
- }
-
- @Override
- public String getActionName() {
- StringBuilder sb = new StringBuilder();
- sb.append("postWatchLockOwner(myNode=").append(myNode).append(", siblingNode=")
- .append(siblingNode).append(", ownerNode=").append(ownerNode).append(")");
- return sb.toString();
- }
- }, promise);
- }
- }, null);
- } else {
- promise.setValue(currentOwner.getLeft());
- }
- }
-
- @Override
- public String getActionName() {
- StringBuilder sb = new StringBuilder();
- sb.append("watchLockOwner(myNode=").append(myNode).append(", siblingNode=")
- .append(siblingNode).append(", ownerNode=").append(ownerNode).append(")");
- return sb.toString();
- }
- }, promise);
- }
-
- class LockWatcher implements Watcher {
-
- // Enforce a epoch number to avoid a race on canceling attempt
- final int epoch;
-
- LockWatcher(int epoch) {
- this.epoch = epoch;
- }
-
- @Override
- public void process(WatchedEvent event) {
- LOG.debug("Received event {} from lock {} at {} : watcher epoch {}, lock epoch {}.",
- new Object[] { event, lockPath, System.currentTimeMillis(), epoch, ZKSessionLock.this.epoch.get() });
- if (event.getType() == Watcher.Event.EventType.None) {
- switch (event.getState()) {
- case SyncConnected:
- break;
- case Expired:
- LOG.info("Session {} is expired for lock {} at {} : watcher epoch {}, lock epoch {}.",
- new Object[] { lockId.getRight(), lockPath, System.currentTimeMillis(),
- epoch, ZKSessionLock.this.epoch.get() });
- handleSessionExpired(epoch);
- break;
- default:
- break;
- }
- } else if (event.getType() == Event.EventType.NodeDeleted) {
- // this handles the case where we have aborted a lock and deleted ourselves but still have a
- // watch on the nextLowestNode. This is a workaround since ZK doesn't support unsub.
- if (!event.getPath().equals(watchedNode)) {
- LOG.warn("{} (watching {}) ignored watched event from {} ",
- new Object[] { lockId, watchedNode, event.getPath() });
- return;
- }
- handleNodeDelete(epoch, event);
- } else {
- LOG.warn("Unexpected ZK event: {}", event.getType().name());
- }
- }
-
- }
-}
[30/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConstants.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConstants.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConstants.java
deleted file mode 100644
index e798a0f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConstants.java
+++ /dev/null
@@ -1,69 +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 java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.data.ACL;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-public class DistributedLogConstants {
- public static final byte[] EMPTY_BYTES = new byte[0];
- public static final String SCHEME_PREFIX = "distributedlog";
- public static final String BACKEND_BK = "bk";
- public static final long INVALID_TXID = -999;
- public static final long EMPTY_LOGSEGMENT_TX_ID = -99;
- public static final long MAX_TXID = Long.MAX_VALUE;
- public static final long SMALL_LOGSEGMENT_THRESHOLD = 10;
- public static final int LOGSEGMENT_NAME_VERSION = 1;
- public static final int FUTURE_TIMEOUT_IMMEDIATE = 0;
- public static final int FUTURE_TIMEOUT_INFINITE = -1;
- public static final long LOCK_IMMEDIATE = FUTURE_TIMEOUT_IMMEDIATE;
- public static final long LOCK_TIMEOUT_INFINITE = FUTURE_TIMEOUT_INFINITE;
- public static final long LOCK_OP_TIMEOUT_DEFAULT = 120;
- public static final long LOCK_REACQUIRE_TIMEOUT_DEFAULT = 120;
- public static final String UNKNOWN_CLIENT_ID = "Unknown-ClientId";
- public static final int LOCAL_REGION_ID = 0;
- public static final long LOGSEGMENT_DEFAULT_STATUS = 0;
- public static final long UNASSIGNED_LOGSEGMENT_SEQNO = 0;
- public static final long UNASSIGNED_SEQUENCE_ID = -1L;
- public static final long FIRST_LOGSEGMENT_SEQNO = 1;
- public static final long UNRESOLVED_LEDGER_ID = -1;
- public static final long LATENCY_WARN_THRESHOLD_IN_MILLIS = TimeUnit.SECONDS.toMillis(1);
- public static final int DL_INTERRUPTED_EXCEPTION_RESULT_CODE = Integer.MIN_VALUE + 1;
- public static final int ZK_CONNECTION_EXCEPTION_RESULT_CODE = Integer.MIN_VALUE + 2;
-
- public static final String ALLOCATION_POOL_NODE = ".allocation_pool";
- // log segment prefix
- public static final String INPROGRESS_LOGSEGMENT_PREFIX = "inprogress";
- public static final String COMPLETED_LOGSEGMENT_PREFIX = "logrecs";
- public static final String DISALLOW_PLACEMENT_IN_REGION_FEATURE_NAME = "disallow_bookie_placement";
- static final byte[] CONTROL_RECORD_CONTENT = "control".getBytes(UTF_8);
- static final byte[] KEEPALIVE_RECORD_CONTENT = "keepalive".getBytes(UTF_8);
-
- // An ACL that gives all permissions to node creators and read permissions only to everyone else.
- public static final List<ACL> EVERYONE_READ_CREATOR_ALL =
- ImmutableList.<ACL>builder()
- .addAll(Ids.CREATOR_ALL_ACL)
- .addAll(Ids.READ_ACL_UNSAFE)
- .build();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
deleted file mode 100644
index 34cfb65..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
+++ /dev/null
@@ -1,308 +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.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;
-
-/**
- * A DistributedLogManager is responsible for managing a single place of storing
- * edit logs. It may correspond to multiple files, a backup node, etc.
- * Even when the actual underlying storage is rolled, or failed and restored,
- * 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 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.
- *
- * @return log segments
- * @throws IOException
- */
- public List<LogSegmentMetadata> getLogSegments() throws IOException;
-
- /**
- * Register <i>listener</i> on log segment updates of this stream.
- *
- * @param listener
- * listener to receive update log segment list.
- */
- public void registerListener(LogSegmentListener listener) throws IOException ;
-
- /**
- * Unregister <i>listener</i> on log segment updates from this stream.
- *
- * @param listener
- * listener to receive update log segment list.
- */
- public void unregisterListener(LogSegmentListener listener);
-
- /**
- * Open async log writer to write records to the log stream.
- *
- * @return result represents the open result
- */
- public Future<AsyncLogWriter> openAsyncLogWriter();
-
- /**
- * Begin writing to the log stream identified by the name
- *
- * @return the writer interface to generate log records
- */
- public LogWriter startLogSegmentNonPartitioned() throws IOException;
-
- /**
- * Begin writing to the log stream identified by the name
- *
- * @return the writer interface to generate log records
- */
- // @Deprecated
- public AsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException;
-
- /**
- * Begin appending to the end of the log stream which is being treated as a sequence of bytes
- *
- * @return the writer interface to generate log records
- */
- public AppendOnlyStreamWriter getAppendOnlyStreamWriter() throws IOException;
-
- /**
- * Get a reader to read a log stream as a sequence of bytes
- *
- * @return the writer interface to generate log records
- */
- public AppendOnlyStreamReader getAppendOnlyStreamReader() throws IOException;
-
- /**
- * Get the input stream starting with fromTxnId for the specified log
- *
- * @param fromTxnId - the first transaction id we want to read
- * @return the stream starting with transaction fromTxnId
- * @throws IOException if a stream cannot be found.
- */
- public LogReader getInputStream(long fromTxnId)
- throws IOException;
-
- public LogReader getInputStream(DLSN fromDLSN) throws IOException;
-
- /**
- * Open an async log reader to read records from a log starting from <code>fromTxnId</code>.
- *
- * @param fromTxnId
- * transaction id to start reading from
- * @return async log reader
- */
- public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId);
-
- /**
- * Open an async log reader to read records from a log starting from <code>fromDLSN</code>
- *
- * @param fromDLSN
- * dlsn to start reading from
- * @return async log reader
- */
- public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN);
-
- // @Deprecated
- public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException;
-
- // @Deprecated
- public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException;
-
- public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
-
- /**
- * Get a log reader with lock starting from <i>fromDLSN</i> and using <i>subscriberId</i>.
- * If two readers tried to open using same subscriberId, one would succeed, while the other
- * will be blocked until it gets the lock.
- *
- * @param fromDLSN
- * start dlsn
- * @param subscriberId
- * subscriber id
- * @return async log reader
- */
- public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN, String subscriberId);
-
- /**
- * Get a log reader using <i>subscriberId</i> with lock. The reader will start reading from
- * its last commit position recorded in subscription store. If no last commit position found
- * in subscription store, it would start reading from head of the stream.
- *
- * If the two readers tried to open using same subscriberId, one would succeed, while the other
- * will be blocked until it gets the lock.
- *
- * @param subscriberId
- * subscriber id
- * @return async log reader
- */
- public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
-
- /**
- * Get the {@link DLSN} of first log record whose transaction id is not less than <code>transactionId</code>.
- *
- * @param transactionId
- * transaction id
- * @return dlsn of first log record whose transaction id is not less than transactionId.
- */
- public Future<DLSN> getDLSNNotLessThanTxId(long transactionId);
-
- /**
- * Get the last log record in the stream
- *
- * @return the last log record in the stream
- * @throws IOException if a stream cannot be found.
- */
- public LogRecordWithDLSN getLastLogRecord()
- throws IOException;
-
- /**
- * Get the earliest Transaction Id available in the log
- *
- * @return earliest transaction id
- * @throws IOException
- */
- public long getFirstTxId() throws IOException;
-
- /**
- * Get Latest Transaction Id in the log
- *
- * @return latest transaction id
- * @throws IOException
- */
- public long getLastTxId() throws IOException;
-
- /**
- * Get Latest DLSN in the log
- *
- * @return last dlsn
- * @throws IOException
- */
- public DLSN getLastDLSN() throws IOException;
-
- /**
- * Get Latest log record with DLSN in the log - async
- *
- * @return latest log record with DLSN
- */
- public Future<LogRecordWithDLSN> getLastLogRecordAsync();
-
- /**
- * Get Latest Transaction Id in the log - async
- *
- * @return latest transaction id
- */
- public Future<Long> getLastTxIdAsync();
-
- /**
- * Get first DLSN in the log.
- *
- * @return first dlsn in the stream
- */
- public Future<DLSN> getFirstDLSNAsync();
-
- /**
- * Get Latest DLSN in the log - async
- *
- * @return latest transaction id
- */
- public Future<DLSN> getLastDLSNAsync();
-
- /**
- * Get the number of log records in the active portion of the log
- * Any log segments that have already been truncated will not be included
- *
- * @return number of log records
- * @throws IOException
- */
- public long getLogRecordCount() throws IOException;
-
- /**
- * Get the number of log records in the active portion of the log - async.
- * Any log segments that have already been truncated will not be included
- *
- * @return future number of log records
- * @throws IOException
- */
- public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN);
-
- /**
- * Run recovery on the log.
- *
- * @throws IOException
- */
- public void recover() throws IOException;
-
- /**
- * Check if an end of stream marker was added to the stream
- * A stream with an end of stream marker cannot be appended to
- *
- * @return true if the marker was added to the stream, false otherwise
- * @throws IOException
- */
- public boolean isEndOfStreamMarked() throws IOException;
-
- /**
- * Delete the log.
- *
- * @throws IOException if the deletion fails
- */
- public void delete() throws IOException;
-
- /**
- * The DistributedLogManager may archive/purge any logs for transactionId
- * less than or equal to minImageTxId.
- * This is to be used only when the client explicitly manages deletion. If
- * the cleanup policy is based on sliding time window, then this method need
- * not be called.
- *
- * @param minTxIdToKeep the earliest txid that must be retained
- * @throws IOException if purging fails
- */
- public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException;
-
- /**
- * Get the subscriptions store provided by the distributedlog manager.
- *
- * @return subscriptions store manages subscriptions for current stream.
- */
- public SubscriptionsStore getSubscriptionsStore();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/Entry.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/Entry.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/Entry.java
deleted file mode 100644
index bf315fc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/Entry.java
+++ /dev/null
@@ -1,403 +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.google.common.base.Preconditions;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import javax.annotation.Nullable;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * A set of {@link LogRecord}s.
- */
-public class Entry {
-
- /**
- * Create a new log record set.
- *
- * @param logName
- * name of the log
- * @param initialBufferSize
- * initial buffer size
- * @param envelopeBeforeTransmit
- * if envelope the buffer before transmit
- * @param codec
- * compression codec
- * @param statsLogger
- * stats logger to receive stats
- * @return writer to build a log record set.
- */
- public static Writer newEntry(
- String logName,
- int initialBufferSize,
- boolean envelopeBeforeTransmit,
- CompressionCodec.Type codec,
- StatsLogger statsLogger) {
- return new EnvelopedEntryWriter(
- logName,
- initialBufferSize,
- envelopeBeforeTransmit,
- codec,
- statsLogger);
- }
-
- public static Builder newBuilder() {
- return new Builder();
- }
-
- /**
- * Build the record set object.
- */
- public static class Builder {
-
- private long logSegmentSequenceNumber = -1;
- private long entryId = -1;
- private long startSequenceId = Long.MIN_VALUE;
- private boolean envelopeEntry = true;
- // input stream
- private InputStream in = null;
- // or bytes array
- private byte[] data = null;
- private int offset = -1;
- private int length = -1;
- private Optional<Long> txidToSkipTo = Optional.absent();
- private Optional<DLSN> dlsnToSkipTo = Optional.absent();
- private boolean deserializeRecordSet = true;
-
- private Builder() {}
-
- /**
- * Reset the builder.
- *
- * @return builder
- */
- public Builder reset() {
- logSegmentSequenceNumber = -1;
- entryId = -1;
- startSequenceId = Long.MIN_VALUE;
- envelopeEntry = true;
- // input stream
- in = null;
- // or bytes array
- data = null;
- offset = -1;
- length = -1;
- txidToSkipTo = Optional.absent();
- dlsnToSkipTo = Optional.absent();
- return this;
- }
-
- /**
- * Set the segment info of the log segment that this record
- * set belongs to.
- *
- * @param lssn
- * log segment sequence number
- * @param startSequenceId
- * start sequence id of this log segment
- * @return builder
- */
- public Builder setLogSegmentInfo(long lssn, long startSequenceId) {
- this.logSegmentSequenceNumber = lssn;
- this.startSequenceId = startSequenceId;
- return this;
- }
-
- /**
- * Set the entry id of this log record set.
- *
- * @param entryId
- * entry id assigned for this log record set.
- * @return builder
- */
- public Builder setEntryId(long entryId) {
- this.entryId = entryId;
- return this;
- }
-
- /**
- * Set whether this record set is enveloped or not.
- *
- * @param enabled
- * flag indicates whether this record set is enveloped or not.
- * @return builder
- */
- public Builder setEnvelopeEntry(boolean enabled) {
- this.envelopeEntry = enabled;
- return this;
- }
-
- /**
- * Set the serialized bytes data of this record set.
- *
- * @param data
- * serialized bytes data of this record set.
- * @param offset
- * offset of the bytes data
- * @param length
- * length of the bytes data
- * @return builder
- */
- public Builder setData(byte[] data, int offset, int length) {
- this.data = data;
- this.offset = offset;
- this.length = length;
- return this;
- }
-
- /**
- * Set the input stream of the serialized bytes data of this record set.
- *
- * @param in
- * input stream
- * @return builder
- */
- public Builder setInputStream(InputStream in) {
- this.in = in;
- return this;
- }
-
- /**
- * Set the record set starts from <code>dlsn</code>.
- *
- * @param dlsn
- * dlsn to skip to
- * @return builder
- */
- public Builder skipTo(@Nullable DLSN dlsn) {
- this.dlsnToSkipTo = Optional.fromNullable(dlsn);
- return this;
- }
-
- /**
- * Set the record set starts from <code>txid</code>.
- *
- * @param txid
- * txid to skip to
- * @return builder
- */
- public Builder skipTo(long txid) {
- this.txidToSkipTo = Optional.of(txid);
- return this;
- }
-
- /**
- * Enable/disable deserialize record set.
- *
- * @param enabled
- * flag to enable/disable dserialize record set.
- * @return builder
- */
- public Builder deserializeRecordSet(boolean enabled) {
- this.deserializeRecordSet = enabled;
- return this;
- }
-
- public Entry build() {
- Preconditions.checkNotNull(data, "Serialized data isn't provided");
- Preconditions.checkArgument(offset >= 0 && length >= 0
- && (offset + length) <= data.length,
- "Invalid offset or length of serialized data");
- return new Entry(
- logSegmentSequenceNumber,
- entryId,
- startSequenceId,
- envelopeEntry,
- deserializeRecordSet,
- data,
- offset,
- length,
- txidToSkipTo,
- dlsnToSkipTo);
- }
-
- public Entry.Reader buildReader() throws IOException {
- Preconditions.checkArgument(data != null || in != null,
- "Serialized data or input stream isn't provided");
- InputStream in;
- if (null != this.in) {
- in = this.in;
- } else {
- Preconditions.checkArgument(offset >= 0 && length >= 0
- && (offset + length) <= data.length,
- "Invalid offset or length of serialized data");
- in = new ByteArrayInputStream(data, offset, length);
- }
- return new EnvelopedEntryReader(
- logSegmentSequenceNumber,
- entryId,
- startSequenceId,
- in,
- envelopeEntry,
- deserializeRecordSet,
- NullStatsLogger.INSTANCE);
- }
-
- }
-
- private final long logSegmentSequenceNumber;
- private final long entryId;
- private final long startSequenceId;
- private final boolean envelopedEntry;
- private final boolean deserializeRecordSet;
- private final byte[] data;
- private final int offset;
- private final int length;
- private final Optional<Long> txidToSkipTo;
- private final Optional<DLSN> dlsnToSkipTo;
-
- private Entry(long logSegmentSequenceNumber,
- long entryId,
- long startSequenceId,
- boolean envelopedEntry,
- boolean deserializeRecordSet,
- byte[] data,
- int offset,
- int length,
- Optional<Long> txidToSkipTo,
- Optional<DLSN> dlsnToSkipTo) {
- this.logSegmentSequenceNumber = logSegmentSequenceNumber;
- this.entryId = entryId;
- this.startSequenceId = startSequenceId;
- this.envelopedEntry = envelopedEntry;
- this.deserializeRecordSet = deserializeRecordSet;
- this.data = data;
- this.offset = offset;
- this.length = length;
- this.txidToSkipTo = txidToSkipTo;
- this.dlsnToSkipTo = dlsnToSkipTo;
- }
-
- /**
- * Get raw data of this record set.
- *
- * @return raw data representation of this record set.
- */
- public byte[] getRawData() {
- return data;
- }
-
- /**
- * Create reader to iterate over this record set.
- *
- * @return reader to iterate over this record set.
- * @throws IOException if the record set is invalid record set.
- */
- public Reader reader() throws IOException {
- InputStream in = new ByteArrayInputStream(data, offset, length);
- Reader reader = new EnvelopedEntryReader(
- logSegmentSequenceNumber,
- entryId,
- startSequenceId,
- in,
- envelopedEntry,
- deserializeRecordSet,
- NullStatsLogger.INSTANCE);
- if (txidToSkipTo.isPresent()) {
- reader.skipTo(txidToSkipTo.get());
- }
- if (dlsnToSkipTo.isPresent()) {
- reader.skipTo(dlsnToSkipTo.get());
- }
- return reader;
- }
-
- /**
- * Writer to append {@link LogRecord}s to {@link Entry}.
- */
- public interface Writer extends EntryBuffer {
-
- /**
- * Write a {@link LogRecord} to this record set.
- *
- * @param record
- * record to write
- * @param transmitPromise
- * callback for transmit result. the promise is only
- * satisfied when this record set is transmitted.
- * @throws LogRecordTooLongException if the record is too long
- * @throws WriteException when encountered exception writing the record
- */
- void writeRecord(LogRecord record, Promise<DLSN> transmitPromise)
- throws LogRecordTooLongException, WriteException;
-
- /**
- * Reset the writer to write records.
- */
- void reset();
-
- }
-
- /**
- * Reader to read {@link LogRecord}s from this record set.
- */
- public interface Reader {
-
- /**
- * Get the log segment sequence number.
- *
- * @return the log segment sequence number.
- */
- long getLSSN();
-
- /**
- * Return the entry id.
- *
- * @return the entry id.
- */
- long getEntryId();
-
- /**
- * Read next log record from this record set.
- *
- * @return next log record from this record set.
- */
- LogRecordWithDLSN nextRecord() throws IOException;
-
- /**
- * Skip the reader to the record whose transaction id is <code>txId</code>.
- *
- * @param txId
- * transaction id to skip to.
- * @return true if skip succeeds, otherwise false.
- * @throws IOException
- */
- boolean skipTo(long txId) throws IOException;
-
- /**
- * Skip the reader to the record whose DLSN is <code>dlsn</code>.
- *
- * @param dlsn
- * DLSN to skip to.
- * @return true if skip succeeds, otherwise false.
- * @throws IOException
- */
- boolean skipTo(DLSN dlsn) throws IOException;
-
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryBuffer.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryBuffer.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryBuffer.java
deleted file mode 100644
index 394fbad..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryBuffer.java
+++ /dev/null
@@ -1,70 +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.InvalidEnvelopedEntryException;
-import com.twitter.distributedlog.io.Buffer;
-import com.twitter.distributedlog.io.TransmitListener;
-
-import java.io.IOException;
-
-/**
- * Write representation of a {@link Entry}.
- * It is a buffer of log record set, used for transmission.
- */
-public interface EntryBuffer extends TransmitListener {
-
- /**
- * Return if this record set contains user records.
- *
- * @return true if this record set contains user records, otherwise
- * return false.
- */
- boolean hasUserRecords();
-
- /**
- * Return number of records in current record set.
- *
- * @return number of records in current record set.
- */
- int getNumRecords();
-
- /**
- * Return number of bytes in current record set.
- *
- * @return number of bytes in current record set.
- */
- int getNumBytes();
-
- /**
- * Return max tx id in current record set.
- *
- * @return max tx id.
- */
- long getMaxTxId();
-
- /**
- * Get the buffer to transmit.
- *
- * @return the buffer to transmit.
- * @throws InvalidEnvelopedEntryException if the record set buffer is invalid
- * @throws IOException when encountered IOException during serialization
- */
- Buffer getBuffer() throws InvalidEnvelopedEntryException, IOException;
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryPosition.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryPosition.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryPosition.java
deleted file mode 100644
index 0a15d29..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryPosition.java
+++ /dev/null
@@ -1,63 +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;
-
-/**
- * The position of an entry, identified by log segment sequence number and entry id.
- */
-class EntryPosition {
-
- private long lssn;
- private long entryId;
-
- EntryPosition(long lssn, long entryId) {
- this.lssn = lssn;
- this.entryId = entryId;
- }
-
- public synchronized long getLogSegmentSequenceNumber() {
- return lssn;
- }
-
- public synchronized long getEntryId() {
- return entryId;
- }
-
- public synchronized boolean advance(long lssn, long entryId) {
- if (lssn == this.lssn) {
- if (entryId <= this.entryId) {
- return false;
- }
- this.entryId = entryId;
- return true;
- } else if (lssn > this.lssn) {
- this.lssn = lssn;
- this.entryId = entryId;
- return true;
- } else {
- return false;
- }
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("(").append(lssn).append(", ").append(entryId).append(")");
- return sb.toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntry.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntry.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntry.java
deleted file mode 100644
index 55d3be9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntry.java
+++ /dev/null
@@ -1,296 +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 java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-import com.google.common.base.Preconditions;
-
-import com.twitter.distributedlog.exceptions.InvalidEnvelopedEntryException;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations.Compression;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.io.CompressionUtils;
-import com.twitter.distributedlog.util.BitMaskUtils;
-
-/**
- * An enveloped entry written to BookKeeper.
- *
- * Data type in brackets. Interpretation should be on the basis of data types and not individual
- * bytes to honor Endianness.
- *
- * Entry Structure:
- * ---------------
- * Bytes 0 : Version (Byte)
- * Bytes 1 - (DATA = 1+Header.length-1) : Header (Integer)
- * Bytes DATA - DATA+3 : Payload Length (Integer)
- * BYTES DATA+4 - DATA+4+payload.length-1 : Payload (Byte[])
- *
- * V1 Header Structure: // Offsets relative to the start of the header.
- * -------------------
- * Bytes 0 - 3 : Flags (Integer)
- * Bytes 4 - 7 : Original payload size before compression (Integer)
- *
- * Flags: // 32 Bits
- * -----
- * 0 ... 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
- * |_|
- * |
- * Compression Type
- *
- * Compression Type: // 2 Bits (Least significant)
- * ----------------
- * 00 : No Compression
- * 01 : LZ4 Compression
- * 10 : Unused
- * 11 : Unused
- */
-public class EnvelopedEntry {
-
- public static final int VERSION_LENGTH = 1; // One byte long
- public static final byte VERSION_ONE = 1;
-
- public static final byte LOWEST_SUPPORTED_VERSION = VERSION_ONE;
- public static final byte HIGHEST_SUPPORTED_VERSION = VERSION_ONE;
- public static final byte CURRENT_VERSION = VERSION_ONE;
-
- private final OpStatsLogger compressionStat;
- private final OpStatsLogger decompressionStat;
- private final Counter compressedEntryBytes;
- private final Counter decompressedEntryBytes;
- private final byte version;
-
- private Header header = new Header();
- private Payload payloadCompressed = new Payload();
- private Payload payloadDecompressed = new Payload();
-
- public EnvelopedEntry(byte version,
- StatsLogger statsLogger) throws InvalidEnvelopedEntryException {
- Preconditions.checkNotNull(statsLogger);
- if (version < LOWEST_SUPPORTED_VERSION || version > HIGHEST_SUPPORTED_VERSION) {
- throw new InvalidEnvelopedEntryException("Invalid enveloped entry version " + version + ", expected to be in [ "
- + LOWEST_SUPPORTED_VERSION + " ~ " + HIGHEST_SUPPORTED_VERSION + " ]");
- }
- this.version = version;
- this.compressionStat = statsLogger.getOpStatsLogger("compression_time");
- this.decompressionStat = statsLogger.getOpStatsLogger("decompression_time");
- this.compressedEntryBytes = statsLogger.getCounter("compressed_bytes");
- this.decompressedEntryBytes = statsLogger.getCounter("decompressed_bytes");
- }
-
- /**
- * @param statsLogger
- * Used for getting stats for (de)compression time
- * @param compressionType
- * The compression type to use
- * @param decompressed
- * The decompressed payload
- * NOTE: The size of the byte array passed as the decompressed payload can be larger
- * than the actual contents to be compressed.
- */
- public EnvelopedEntry(byte version,
- CompressionCodec.Type compressionType,
- byte[] decompressed,
- int length,
- StatsLogger statsLogger)
- throws InvalidEnvelopedEntryException {
- this(version, statsLogger);
- Preconditions.checkNotNull(compressionType);
- Preconditions.checkNotNull(decompressed);
- Preconditions.checkArgument(length >= 0, "Invalid bytes length " + length);
-
- this.header = new Header(compressionType, length);
- this.payloadDecompressed = new Payload(length, decompressed);
- }
-
- private boolean isReady() {
- return (header.ready && payloadDecompressed.ready);
- }
-
- @Compression
- public void writeFully(DataOutputStream out) throws IOException {
- Preconditions.checkNotNull(out);
- if (!isReady()) {
- throw new IOException("Entry not writable");
- }
- // Version
- out.writeByte(version);
- // Header
- header.write(out);
- // Compress
- CompressionCodec codec = CompressionUtils.getCompressionCodec(header.compressionType);
- byte[] compressed = codec.compress(
- payloadDecompressed.payload,
- 0,
- payloadDecompressed.length,
- compressionStat);
- this.payloadCompressed = new Payload(compressed.length, compressed);
- this.compressedEntryBytes.add(payloadCompressed.length);
- this.decompressedEntryBytes.add(payloadDecompressed.length);
- payloadCompressed.write(out);
- }
-
- @Compression
- public void readFully(DataInputStream in) throws IOException {
- Preconditions.checkNotNull(in);
- // Make sure we're reading the right versioned entry.
- byte version = in.readByte();
- if (version != this.version) {
- throw new IOException(String.format("Version mismatch while reading. Received: %d," +
- " Required: %d", version, this.version));
- }
- header.read(in);
- payloadCompressed.read(in);
- // Decompress
- CompressionCodec codec = CompressionUtils.getCompressionCodec(header.compressionType);
- byte[] decompressed = codec.decompress(
- payloadCompressed.payload,
- 0,
- payloadCompressed.length,
- header.decompressedSize,
- decompressionStat);
- this.payloadDecompressed = new Payload(decompressed.length, decompressed);
- this.compressedEntryBytes.add(payloadCompressed.length);
- this.decompressedEntryBytes.add(payloadDecompressed.length);
- }
-
- public byte[] getDecompressedPayload() throws IOException {
- if (!isReady()) {
- throw new IOException("Decompressed payload is not initialized");
- }
- return payloadDecompressed.payload;
- }
-
- public static class Header {
- public static final int COMPRESSION_CODEC_MASK = 0x3;
- public static final int COMPRESSION_CODEC_NONE = 0x0;
- public static final int COMPRESSION_CODEC_LZ4 = 0x1;
-
- private int flags = 0;
- private int decompressedSize = 0;
- private CompressionCodec.Type compressionType = CompressionCodec.Type.UNKNOWN;
-
- // Whether this struct is ready for reading/writing.
- private boolean ready = false;
-
- // Used while reading.
- public Header() {
- }
-
- public Header(CompressionCodec.Type compressionType,
- int decompressedSize) {
- this.compressionType = compressionType;
- this.decompressedSize = decompressedSize;
- this.flags = 0;
- switch (compressionType) {
- case NONE:
- this.flags = (int) BitMaskUtils.set(flags, COMPRESSION_CODEC_MASK,
- COMPRESSION_CODEC_NONE);
- break;
- case LZ4:
- this.flags = (int) BitMaskUtils.set(flags, COMPRESSION_CODEC_MASK,
- COMPRESSION_CODEC_LZ4);
- break;
- default:
- throw new RuntimeException(String.format("Unknown Compression Type: %s",
- compressionType));
- }
- // This can now be written.
- this.ready = true;
- }
-
- private void write(DataOutputStream out) throws IOException {
- out.writeInt(flags);
- out.writeInt(decompressedSize);
- }
-
- private void read(DataInputStream in) throws IOException {
- this.flags = in.readInt();
- int compressionType = (int) BitMaskUtils.get(flags, COMPRESSION_CODEC_MASK);
- if (compressionType == COMPRESSION_CODEC_NONE) {
- this.compressionType = CompressionCodec.Type.NONE;
- } else if (compressionType == COMPRESSION_CODEC_LZ4) {
- this.compressionType = CompressionCodec.Type.LZ4;
- } else {
- throw new IOException(String.format("Unsupported Compression Type: %s",
- compressionType));
- }
- this.decompressedSize = in.readInt();
- // Values can now be read.
- this.ready = true;
- }
- }
-
- public static class Payload {
- private int length = 0;
- private byte[] payload = null;
-
- // Whether this struct is ready for reading/writing.
- private boolean ready = false;
-
- // Used for reading
- Payload() {
- }
-
- Payload(int length, byte[] payload) {
- this.length = length;
- this.payload = payload;
- this.ready = true;
- }
-
- private void write(DataOutputStream out) throws IOException {
- out.writeInt(length);
- out.write(payload, 0, length);
- }
-
- private void read(DataInputStream in) throws IOException {
- this.length = in.readInt();
- this.payload = new byte[length];
- in.readFully(payload);
- this.ready = true;
- }
- }
-
- /**
- * Return an InputStream that reads from the provided InputStream, decompresses the data
- * and returns a new InputStream wrapping the underlying payload.
- *
- * Note that src is modified by this call.
- *
- * @return
- * New Input stream with the underlying payload.
- * @throws Exception
- */
- public static InputStream fromInputStream(InputStream src,
- StatsLogger statsLogger) throws IOException {
- src.mark(VERSION_LENGTH);
- byte version = new DataInputStream(src).readByte();
- src.reset();
- EnvelopedEntry entry = new EnvelopedEntry(version, statsLogger);
- entry.readFully(new DataInputStream(src));
- return new ByteArrayInputStream(entry.getDecompressedPayload());
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryReader.java
deleted file mode 100644
index 038bb18..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryReader.java
+++ /dev/null
@@ -1,102 +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 org.apache.bookkeeper.stats.StatsLogger;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * Record reader to read records from an enveloped entry buffer.
- */
-class EnvelopedEntryReader implements Entry.Reader, RecordStream {
-
- private final long logSegmentSeqNo;
- private final long entryId;
- private final LogRecord.Reader reader;
-
- // slot id
- private long slotId = 0;
-
- EnvelopedEntryReader(long logSegmentSeqNo,
- long entryId,
- long startSequenceId,
- InputStream in,
- boolean envelopedEntry,
- boolean deserializeRecordSet,
- StatsLogger statsLogger)
- throws IOException {
- this.logSegmentSeqNo = logSegmentSeqNo;
- this.entryId = entryId;
- InputStream src = in;
- if (envelopedEntry) {
- src = EnvelopedEntry.fromInputStream(in, statsLogger);
- }
- this.reader = new LogRecord.Reader(
- this,
- new DataInputStream(src),
- startSequenceId,
- deserializeRecordSet);
- }
-
- @Override
- public long getLSSN() {
- return logSegmentSeqNo;
- }
-
- @Override
- public long getEntryId() {
- return entryId;
- }
-
- @Override
- public LogRecordWithDLSN nextRecord() throws IOException {
- return reader.readOp();
- }
-
- @Override
- public boolean skipTo(long txId) throws IOException {
- return reader.skipTo(txId, true);
- }
-
- @Override
- public boolean skipTo(DLSN dlsn) throws IOException {
- return reader.skipTo(dlsn);
- }
-
- //
- // Record Stream
- //
-
- @Override
- public void advance(int numRecords) {
- slotId += numRecords;
- }
-
- @Override
- public DLSN getCurrentPosition() {
- return new DLSN(logSegmentSeqNo, entryId, slotId);
- }
-
- @Override
- public String getName() {
- return "EnvelopedReader";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryWriter.java
deleted file mode 100644
index 01a91ab..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryWriter.java
+++ /dev/null
@@ -1,192 +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.Entry.Writer;
-import com.twitter.distributedlog.exceptions.InvalidEnvelopedEntryException;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.WriteCancelledException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.io.Buffer;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-
-/**
- * {@link com.twitter.distributedlog.io.Buffer} based log record set writer.
- */
-class EnvelopedEntryWriter implements Writer {
-
- static final Logger logger = LoggerFactory.getLogger(EnvelopedEntryWriter.class);
-
- private static class WriteRequest {
-
- private final int numRecords;
- private final Promise<DLSN> promise;
-
- WriteRequest(int numRecords, Promise<DLSN> promise) {
- this.numRecords = numRecords;
- this.promise = promise;
- }
-
- }
-
- private final String logName;
- private final Buffer buffer;
- private final LogRecord.Writer writer;
- private final List<WriteRequest> writeRequests;
- private final boolean envelopeBeforeTransmit;
- private final CompressionCodec.Type codec;
- private final StatsLogger statsLogger;
- private int count = 0;
- private boolean hasUserData = false;
- private long maxTxId = Long.MIN_VALUE;
-
- EnvelopedEntryWriter(String logName,
- int initialBufferSize,
- boolean envelopeBeforeTransmit,
- CompressionCodec.Type codec,
- StatsLogger statsLogger) {
- this.logName = logName;
- this.buffer = new Buffer(initialBufferSize * 6 / 5);
- this.writer = new LogRecord.Writer(new DataOutputStream(buffer));
- this.writeRequests = new LinkedList<WriteRequest>();
- this.envelopeBeforeTransmit = envelopeBeforeTransmit;
- this.codec = codec;
- this.statsLogger = statsLogger;
- }
-
- @Override
- public synchronized void reset() {
- cancelPromises(new WriteCancelledException(logName, "Record Set is reset"));
- count = 0;
- this.buffer.reset();
- }
-
- @Override
- public synchronized void writeRecord(LogRecord record,
- Promise<DLSN> transmitPromise)
- throws LogRecordTooLongException, WriteException {
- int logRecordSize = record.getPersistentSize();
- if (logRecordSize > MAX_LOGRECORD_SIZE) {
- throw new LogRecordTooLongException(
- "Log Record of size " + logRecordSize + " written when only "
- + MAX_LOGRECORD_SIZE + " is allowed");
- }
-
- try {
- this.writer.writeOp(record);
- int numRecords = 1;
- if (!record.isControl()) {
- hasUserData = true;
- }
- if (record.isRecordSet()) {
- numRecords = LogRecordSet.numRecords(record);
- }
- count += numRecords;
- writeRequests.add(new WriteRequest(numRecords, transmitPromise));
- maxTxId = Math.max(maxTxId, record.getTransactionId());
- } catch (IOException e) {
- logger.error("Failed to append record to record set of {} : ",
- logName, e);
- throw new WriteException(logName, "Failed to append record to record set of "
- + logName);
- }
- }
-
- private synchronized void satisfyPromises(long lssn, long entryId) {
- long nextSlotId = 0;
- for (WriteRequest request : writeRequests) {
- request.promise.setValue(new DLSN(lssn, entryId, nextSlotId));
- nextSlotId += request.numRecords;
- }
- writeRequests.clear();
- }
-
- private synchronized void cancelPromises(Throwable reason) {
- for (WriteRequest request : writeRequests) {
- request.promise.setException(reason);
- }
- writeRequests.clear();
- }
-
- @Override
- public synchronized long getMaxTxId() {
- return maxTxId;
- }
-
- @Override
- public synchronized boolean hasUserRecords() {
- return hasUserData;
- }
-
- @Override
- public int getNumBytes() {
- return buffer.size();
- }
-
- @Override
- public synchronized int getNumRecords() {
- return count;
- }
-
- @Override
- public synchronized Buffer getBuffer() throws InvalidEnvelopedEntryException, IOException {
- if (!envelopeBeforeTransmit) {
- return buffer;
- }
- // We can't escape this allocation because things need to be read from one byte array
- // and then written to another. This is the destination.
- Buffer toSend = new Buffer(buffer.size());
- byte[] decompressed = buffer.getData();
- int length = buffer.size();
- EnvelopedEntry entry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
- codec,
- decompressed,
- length,
- statsLogger);
- // This will cause an allocation of a byte[] for compression. This can be avoided
- // but we can do that later only if needed.
- entry.writeFully(new DataOutputStream(toSend));
- return toSend;
- }
-
- @Override
- public synchronized DLSN finalizeTransmit(long lssn, long entryId) {
- return new DLSN(lssn, entryId, count - 1);
- }
-
- @Override
- public void completeTransmit(long lssn, long entryId) {
- satisfyPromises(lssn, entryId);
- }
-
- @Override
- public void abortTransmit(Throwable reason) {
- cancelPromises(reason);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LedgerReadPosition.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LedgerReadPosition.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LedgerReadPosition.java
deleted file mode 100644
index 550d314..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LedgerReadPosition.java
+++ /dev/null
@@ -1,171 +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 java.io.Serializable;
-import java.util.Comparator;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class LedgerReadPosition {
- static final Logger LOG = LoggerFactory.getLogger(LedgerReadPosition.class);
-
- private static enum PartialOrderingComparisonResult {
- NotComparable,
- GreaterThan,
- LessThan,
- EqualTo
- }
-
- long ledgerId = DistributedLogConstants.UNRESOLVED_LEDGER_ID;
- long logSegmentSequenceNo;
- long entryId;
-
- public LedgerReadPosition(long ledgerId, long logSegmentSequenceNo, long entryId) {
- this.ledgerId = ledgerId;
- this.logSegmentSequenceNo = logSegmentSequenceNo;
- this.entryId = entryId;
- }
-
- public LedgerReadPosition(LedgerReadPosition that) {
- this.ledgerId = that.ledgerId;
- this.logSegmentSequenceNo = that.logSegmentSequenceNo;
- this.entryId = that.entryId;
- }
-
-
- public LedgerReadPosition(final DLSN dlsn) {
- this(dlsn.getLogSegmentSequenceNo(), dlsn.getEntryId());
- }
-
- public LedgerReadPosition(long logSegmentSequenceNo, long entryId) {
- this.logSegmentSequenceNo = logSegmentSequenceNo;
- this.entryId = entryId;
- }
-
- public long getLedgerId() {
- if (DistributedLogConstants.UNRESOLVED_LEDGER_ID == ledgerId) {
- LOG.trace("Ledger Id is not initialized");
- throw new IllegalStateException("Ledger Id is not initialized");
- }
- return ledgerId;
- }
-
- public long getLogSegmentSequenceNumber() {
- return logSegmentSequenceNo;
- }
-
- public long getEntryId() {
- return entryId;
- }
-
- public void advance() {
- entryId++;
- }
-
- public void positionOnNewLogSegment(long ledgerId, long logSegmentSequenceNo) {
- this.ledgerId = ledgerId;
- this.logSegmentSequenceNo = logSegmentSequenceNo;
- this.entryId = 0L;
- }
-
- @Override
- public String toString() {
- return String.format("(lid=%d, lseqNo=%d, eid=%d)", ledgerId, logSegmentSequenceNo, entryId);
- }
-
- public boolean definitelyLessThanOrEqualTo(LedgerReadPosition threshold) {
- PartialOrderingComparisonResult result = comparePartiallyOrdered(threshold);
- return ((result == PartialOrderingComparisonResult.LessThan) ||
- (result == PartialOrderingComparisonResult.EqualTo));
- }
-
- public boolean definitelyLessThan(LedgerReadPosition threshold) {
- PartialOrderingComparisonResult result = comparePartiallyOrdered(threshold);
- return result == PartialOrderingComparisonResult.LessThan;
- }
-
- private PartialOrderingComparisonResult comparePartiallyOrdered(LedgerReadPosition threshold) {
- // If no threshold is passed we cannot make a definitive comparison
- if (null == threshold) {
- return PartialOrderingComparisonResult.NotComparable;
- }
-
- if (this.logSegmentSequenceNo != threshold.logSegmentSequenceNo) {
- if (this.logSegmentSequenceNo < threshold.logSegmentSequenceNo) {
- return PartialOrderingComparisonResult.LessThan;
- } else {
- return PartialOrderingComparisonResult.GreaterThan;
- }
- } else if (this.ledgerId != threshold.ledgerId) {
- // When logSegmentSequenceNo is equal we cannot definitely say that this
- // position is less than the threshold unless ledgerIds are equal
- // since LogSegmentSequenceNumber maybe inferred from transactionIds in older
- // versions of the metadata.
- return PartialOrderingComparisonResult.NotComparable;
- } else if (this.getEntryId() < threshold.getEntryId()) {
- return PartialOrderingComparisonResult.LessThan;
- } else if (this.getEntryId() > threshold.getEntryId()) {
- return PartialOrderingComparisonResult.GreaterThan;
- } else {
- return PartialOrderingComparisonResult.EqualTo;
- }
- }
-
- /**
- * Comparator for the key portion
- */
- public static final ReadAheadCacheKeyComparator COMPARATOR = new ReadAheadCacheKeyComparator();
-
- // Only compares the key portion
- @Override
- public boolean equals(Object other) {
- if (!(other instanceof LedgerReadPosition)) {
- return false;
- }
- LedgerReadPosition key = (LedgerReadPosition) other;
- return ledgerId == key.ledgerId &&
- entryId == key.entryId;
- }
-
- @Override
- public int hashCode() {
- return (int) (ledgerId * 13 ^ entryId * 17);
- }
-
- /**
- * Compare EntryKey.
- */
- protected static class ReadAheadCacheKeyComparator implements Comparator<LedgerReadPosition>, Serializable {
-
- private static final long serialVersionUID = 0L;
-
- @Override
- public int compare(LedgerReadPosition left, LedgerReadPosition right) {
- long ret = left.ledgerId - right.ledgerId;
- if (ret == 0) {
- ret = left.entryId - right.entryId;
- }
- return (ret < 0) ? -1 : ((ret > 0) ? 1 : 0);
- }
- }
-
-}
-
-
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
deleted file mode 100644
index f4a1e41..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
+++ /dev/null
@@ -1,364 +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.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.DLMetadata;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.proto.BookieServer;
-import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.bookkeeper.util.LocalBookKeeper;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.BindException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Utility class for setting up bookkeeper ensembles
- * and bringing individual bookies up and down
- */
-public class LocalDLMEmulator {
- private static final Logger LOG = LoggerFactory.getLogger(LocalDLMEmulator.class);
-
- public static final String DLOG_NAMESPACE = "/messaging/distributedlog";
-
- private static final int DEFAULT_BOOKIE_INITIAL_PORT = 0; // Use ephemeral ports
- private static final int DEFAULT_ZK_TIMEOUT_SEC = 10;
- private static final int DEFAULT_ZK_PORT = 2181;
- private static final String DEFAULT_ZK_HOST = "127.0.0.1";
- private static final String DEFAULT_ZK_ENSEMBLE = DEFAULT_ZK_HOST + ":" + DEFAULT_ZK_PORT;
- private static final int DEFAULT_NUM_BOOKIES = 3;
- private static final ServerConfiguration DEFAULT_SERVER_CONFIGURATION = new ServerConfiguration();
-
- private final String zkEnsemble;
- private final URI uri;
- private final List<File> tmpDirs = new ArrayList<File>();
- private final int zkTimeoutSec;
- private final Thread bkStartupThread;
- private final String zkHost;
- private final int zkPort;
- private final int numBookies;
-
- public static class Builder {
- private int zkTimeoutSec = DEFAULT_ZK_TIMEOUT_SEC;
- private int numBookies = DEFAULT_NUM_BOOKIES;
- private String zkHost = DEFAULT_ZK_HOST;
- private int zkPort = DEFAULT_ZK_PORT;
- private int initialBookiePort = DEFAULT_BOOKIE_INITIAL_PORT;
- private boolean shouldStartZK = true;
- private Optional<ServerConfiguration> serverConf = Optional.absent();
-
- public Builder numBookies(int numBookies) {
- this.numBookies = numBookies;
- return this;
- }
- public Builder zkHost(String zkHost) {
- this.zkHost = zkHost;
- return this;
- }
- public Builder zkPort(int zkPort) {
- this.zkPort = zkPort;
- return this;
- }
- public Builder zkTimeoutSec(int zkTimeoutSec) {
- this.zkTimeoutSec = zkTimeoutSec;
- return this;
- }
- public Builder initialBookiePort(int initialBookiePort) {
- this.initialBookiePort = initialBookiePort;
- return this;
- }
- public Builder shouldStartZK(boolean shouldStartZK) {
- this.shouldStartZK = shouldStartZK;
- return this;
- }
- public Builder serverConf(ServerConfiguration serverConf) {
- this.serverConf = Optional.of(serverConf);
- return this;
- }
-
- public LocalDLMEmulator build() throws Exception {
- ServerConfiguration conf = null;
- if (serverConf.isPresent()) {
- conf = serverConf.get();
- } else {
- conf = (ServerConfiguration) DEFAULT_SERVER_CONFIGURATION.clone();
- conf.setZkTimeout(zkTimeoutSec * 1000);
- }
- ServerConfiguration newConf = new ServerConfiguration();
- newConf.loadConf(conf);
- newConf.setAllowLoopback(true);
-
- return new LocalDLMEmulator(numBookies, shouldStartZK, zkHost, zkPort,
- initialBookiePort, zkTimeoutSec, newConf);
- }
- }
-
- public static Builder newBuilder() {
- return new Builder();
- }
-
- private LocalDLMEmulator(final int numBookies, final boolean shouldStartZK, final String zkHost, final int zkPort, final int initialBookiePort, final int zkTimeoutSec, final ServerConfiguration serverConf) throws Exception {
- this.numBookies = numBookies;
- this.zkHost = zkHost;
- this.zkPort = zkPort;
- this.zkEnsemble = zkHost + ":" + zkPort;
- this.uri = URI.create("distributedlog://" + zkEnsemble + DLOG_NAMESPACE);
- this.zkTimeoutSec = zkTimeoutSec;
- this.bkStartupThread = new Thread() {
- public void run() {
- try {
- LOG.info("Starting {} bookies : allowLoopback = {}", numBookies, serverConf.getAllowLoopback());
- LocalBookKeeper.startLocalBookies(zkHost, zkPort, numBookies, shouldStartZK, initialBookiePort, serverConf);
- LOG.info("{} bookies are started.");
- } catch (InterruptedException e) {
- // go away quietly
- } catch (Exception e) {
- LOG.error("Error starting local bk", e);
- }
- }
- };
- }
-
- public void start() throws Exception {
- bkStartupThread.start();
- if (!LocalBookKeeper.waitForServerUp(zkEnsemble, zkTimeoutSec*1000)) {
- throw new Exception("Error starting zookeeper/bookkeeper");
- }
- int bookiesUp = checkBookiesUp(numBookies, zkTimeoutSec);
- assert (numBookies == bookiesUp);
- // Provision "/messaging/distributedlog" namespace
- DLMetadata.create(new BKDLConfig(zkEnsemble, "/ledgers")).create(uri);
- }
-
- public void teardown() throws Exception {
- if (bkStartupThread != null) {
- bkStartupThread.interrupt();
- bkStartupThread.join();
- }
- for (File dir : tmpDirs) {
- FileUtils.deleteDirectory(dir);
- }
- }
-
- public String getZkServers() {
- return zkEnsemble;
- }
-
- public URI getUri() {
- return uri;
- }
-
- public BookieServer newBookie() throws Exception {
- ServerConfiguration bookieConf = new ServerConfiguration();
- bookieConf.setZkTimeout(zkTimeoutSec * 1000);
- bookieConf.setBookiePort(0);
- bookieConf.setAllowLoopback(true);
- File tmpdir = File.createTempFile("bookie" + UUID.randomUUID() + "_",
- "test");
- if (!tmpdir.delete()) {
- LOG.debug("Fail to delete tmpdir " + tmpdir);
- }
- if (!tmpdir.mkdir()) {
- throw new IOException("Fail to create tmpdir " + tmpdir);
- }
- tmpDirs.add(tmpdir);
-
- bookieConf.setZkServers(zkEnsemble);
- bookieConf.setJournalDirName(tmpdir.getPath());
- bookieConf.setLedgerDirNames(new String[]{tmpdir.getPath()});
-
- BookieServer b = new BookieServer(bookieConf);
- b.start();
- for (int i = 0; i < 10 && !b.isRunning(); i++) {
- Thread.sleep(10000);
- }
- if (!b.isRunning()) {
- throw new IOException("Bookie would not start");
- }
- return b;
- }
-
- /**
- * Check that a number of bookies are available
- *
- * @param count number of bookies required
- * @param timeout number of seconds to wait for bookies to start
- * @throws java.io.IOException if bookies are not started by the time the timeout hits
- */
- public int checkBookiesUp(int count, int timeout) throws Exception {
- ZooKeeper zkc = connectZooKeeper(zkHost, zkPort, zkTimeoutSec);
- try {
- int mostRecentSize = 0;
- for (int i = 0; i < timeout; i++) {
- try {
- List<String> children = zkc.getChildren("/ledgers/available",
- false);
- children.remove("readonly");
- mostRecentSize = children.size();
- if ((mostRecentSize > count) || LOG.isDebugEnabled()) {
- LOG.info("Found " + mostRecentSize + " bookies up, "
- + "waiting for " + count);
- if ((mostRecentSize > count) || LOG.isTraceEnabled()) {
- for (String child : children) {
- LOG.info(" server: " + child);
- }
- }
- }
- if (mostRecentSize == count) {
- break;
- }
- } catch (KeeperException e) {
- // ignore
- }
- Thread.sleep(1000);
- }
- return mostRecentSize;
- } finally {
- zkc.close();
- }
- }
-
- public static String getBkLedgerPath() {
- return "/ledgers";
- }
-
- public static ZooKeeper connectZooKeeper(String zkHost, int zkPort)
- throws IOException, KeeperException, InterruptedException {
- return connectZooKeeper(zkHost, zkPort, DEFAULT_ZK_TIMEOUT_SEC);
- }
-
- public static ZooKeeper connectZooKeeper(String zkHost, int zkPort, int zkTimeoutSec)
- throws IOException, KeeperException, InterruptedException {
- final CountDownLatch latch = new CountDownLatch(1);
- final String zkHostPort = zkHost + ":" + zkPort;
-
- ZooKeeper zkc = new ZooKeeper(zkHostPort, zkTimeoutSec * 1000, new Watcher() {
- public void process(WatchedEvent event) {
- if (event.getState() == Event.KeeperState.SyncConnected) {
- latch.countDown();
- }
- }
- });
- if (!latch.await(zkTimeoutSec, TimeUnit.SECONDS)) {
- throw new IOException("Zookeeper took too long to connect");
- }
- return zkc;
- }
-
- public static URI createDLMURI(String path) throws Exception {
- return createDLMURI(DEFAULT_ZK_ENSEMBLE, path);
- }
-
- public static URI createDLMURI(String zkServers, String path) throws Exception {
- return URI.create("distributedlog://" + zkServers + DLOG_NAMESPACE + path);
- }
-
- /**
- * Try to start zookkeeper locally on any port.
- */
- public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(File zkDir) throws Exception {
- return runZookeeperOnAnyPort((int) (Math.random()*10000+7000), zkDir);
- }
-
- /**
- * Try to start zookkeeper locally on any port beginning with some base port.
- * Dump some socket info when bind fails.
- */
- public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(int basePort, File zkDir) throws Exception {
-
- final int MAX_RETRIES = 20;
- final int MIN_PORT = 1025;
- final int MAX_PORT = 65535;
- ZooKeeperServerShim zks = null;
- int zkPort = basePort;
- boolean success = false;
- int retries = 0;
-
- while (!success) {
- try {
- LOG.info("zk trying to bind to port " + zkPort);
- zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkDir);
- success = true;
- } catch (BindException be) {
- retries++;
- if (retries > MAX_RETRIES) {
- throw be;
- }
- zkPort++;
- if (zkPort > MAX_PORT) {
- zkPort = MIN_PORT;
- }
- }
- }
-
- return Pair.of(zks, zkPort);
- }
-
- public static void main(String[] args) throws Exception {
- try {
- if (args.length < 1) {
- System.out.println("Usage: LocalDLEmulator <zk_port>");
- System.exit(-1);
- }
-
- final int zkPort = Integer.parseInt(args[0]);
- final File zkDir = IOUtils.createTempDir("distrlog", "zookeeper");
- final LocalDLMEmulator localDlm = LocalDLMEmulator.newBuilder()
- .zkPort(zkPort)
- .build();
-
- Runtime.getRuntime().addShutdownHook(new Thread() {
- @Override
- public void run() {
- try {
- localDlm.teardown();
- FileUtils.deleteDirectory(zkDir);
- System.out.println("ByeBye!");
- } catch (Exception e) {
- // do nothing
- }
- }
- });
- localDlm.start();
-
- System.out.println(String.format(
- "DistributedLog Sandbox is running now. You could access distributedlog://%s:%s",
- DEFAULT_ZK_HOST,
- zkPort));
- } catch (Exception ex) {
- System.out.println("Exception occurred running emulator " + ex);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LogReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LogReader.java
deleted file mode 100644
index c12de29..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogReader.java
+++ /dev/null
@@ -1,195 +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.io.AsyncCloseable;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-
-/**
- * <i>LogReader</i> is a `synchronous` reader reading records from a DL log.
- *
- * <h3>Lifecycle of a Reader</h3>
- *
- * A reader is a <i>sequential</i> reader that read records from a DL log starting
- * from a given position. The position could be a <i>DLSN</i> (via {@link DistributedLogManager#getInputStream(DLSN)}
- * or a <i>Transaction ID</i> (via {@link DistributedLogManager#getInputStream(long)}.
- * <p>
- * After the reader is open, it could call {@link #readNext(boolean)} or {@link #readBulk(boolean, int)}
- * to read records out the log from provided position.
- * <p>
- * Closing the reader (via {@link #close()} will release all the resources occupied
- * by this reader instance.
- * <p>
- * Exceptions could be thrown during reading records. Once the exception is thrown,
- * the reader is set to an error state and it isn't usable anymore. It is the application's
- * responsibility to handle the exceptions and re-create readers if necessary.
- * <p>
- * Example:
- * <pre>
- * DistributedLogManager dlm = ...;
- * long nextTxId = ...;
- * LogReader reader = dlm.getInputStream(nextTxId);
- *
- * while (true) { // keep reading & processing records
- * LogRecord record;
- * try {
- * record = reader.readNext(false);
- * nextTxId = record.getTransactionId();
- * // process the record
- * ...
- * } catch (IOException ioe) {
- * // handle the exception
- * ...
- * reader = dlm.getInputStream(nextTxId + 1);
- * }
- * }
- *
- * </pre>
- *
- * <h3>Read Records</h3>
- *
- * Reading records from an <i>endless</i> log in `synchronous` way isn't as
- * trivial as in `asynchronous` way (via {@link AsyncLogReader}. Because it
- * lacks of callback mechanism. LogReader introduces a flag `nonBlocking` on
- * controlling the <i>waiting</i> behavior on `synchronous` reads.
- *
- * <h4>Blocking vs NonBlocking</h4>
- *
- * <i>Blocking</i> (nonBlocking = false) means the reads will wait for records
- * before returning read calls. While <i>NonBlocking</i> (nonBlocking = true)
- * means the reads will only check readahead cache and return whatever records
- * available in the readahead cache.
- * <p>
- * The <i>waiting</i> period varies in <i>blocking</i> mode. If the reader is
- * catching up with writer (there are records in the log), the read call will
- * wait until records are read and returned. If the reader is caught up with
- * writer (there are no more records in the log at read time), the read call
- * will wait for a small period of time (defined in
- * {@link DistributedLogConfiguration#getReadAheadWaitTime()} and return whatever
- * records available in the readahead cache. In other words, if a reader sees
- * no record on blocking reads, it means the reader is `caught-up` with the
- * writer.
- * <p>
- * <i>Blocking</i> and <i>NonBlocking</i> modes are useful for building replicated
- * state machines. Applications could use <i>blocking</i> reads till caught up
- * with latest data. Once they are caught up with latest data, they could start
- * serving their service and turn to <i>non-blocking</i> read mode and tail read
- * data from the logs.
- * <p>
- * See examples below.
- *
- * <h4>Read Single Record</h4>
- *
- * {@link #readNext(boolean)} is reading individual records from a DL log.
- *
- * <pre>
- * LogReader reader = ...
- *
- * // keep reading records in blocking way until no records available in the log
- * LogRecord record = reader.readNext(false);
- * while (null != record) {
- * // process the record
- * ...
- * // read next record
- * records = reader.readNext(false);
- * }
- *
- * ...
- *
- * // reader is caught up with writer, doing non-blocking reads to tail the log
- * while (true) {
- * record = reader.readNext(true)
- * // process the new records
- * ...
- * }
- * </pre>
- *
- * <h4>Read Batch of Records</h4>
- *
- * {@link #readBulk(boolean, int)} is a convenient way to read a batch of records
- * from a DL log.
- *
- * <pre>
- * LogReader reader = ...
- * int N = 10;
- *
- * // keep reading N records in blocking way until no records available in the log
- * List<LogRecord> records = reader.readBulk(false, N);
- * while (!records.isEmpty()) {
- * // process the list of records
- * ...
- * if (records.size() < N) { // no more records available in the log
- * break;
- * }
- * // read next N records
- * records = reader.readBulk(false, N);
- * }
- *
- * ...
- *
- * // reader is caught up with writer, doing non-blocking reads to tail the log
- * while (true) {
- * records = reader.readBulk(true, N)
- * // process the new records
- * ...
- * }
- *
- * </pre>
- *
- * @see AsyncLogReader
- *
- * NOTE:
- * 1. Extending {@link AsyncCloseable}: BKSyncLogReader is implemented based on BKAsyncLogReader, exposing
- * the {@link AsyncCloseable} interface so the reader could be closed asynchronously
- */
-public interface LogReader extends Closeable, AsyncCloseable {
-
- /**
- * Read the next log record from the stream.
- * <p>
- * If <i>nonBlocking</i> is set to true, the call returns immediately by just polling
- * records from read ahead cache. It would return <i>null</i> if there isn't any records
- * available in the read ahead cache.
- * <p>
- * If <i>nonBlocking</i> is set to false, it would does blocking call. The call will
- * block until return a record if there are records in the stream (aka catching up).
- * Otherwise it would wait up to {@link DistributedLogConfiguration#getReadAheadWaitTime()}
- * milliseconds and return null if there isn't any more records in the stream.
- *
- * @param nonBlocking should the read make blocking calls to the backend or rely on the
- * readAhead cache
- * @return an operation from the stream or null if at end of stream
- * @throws IOException if there is an error reading from the stream
- */
- public LogRecordWithDLSN readNext(boolean nonBlocking) throws IOException;
-
- /**
- * Read the next <i>numLogRecords</i> log records from the stream
- *
- * @param nonBlocking should the read make blocking calls to the backend or rely on the
- * readAhead cache
- * @param numLogRecords maximum number of log records returned by this call.
- * @return an operation from the stream or empty list if at end of stream
- * @throws IOException if there is an error reading from the stream
- * @see #readNext(boolean)
- */
- public List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords) throws IOException;
-}
[03/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
new file mode 100644
index 0000000..0b8c55a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
@@ -0,0 +1,992 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
+import com.google.common.base.Ticker;
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.EndOfLogSegmentException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.logsegment.LogSegmentEntryReader;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.logsegment.LogSegmentFilter;
+import org.apache.distributedlog.util.OrderedScheduler;
+import com.twitter.util.Function0;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Futures;
+import com.twitter.util.Promise;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Function1;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * New ReadAhead Reader that uses {@link org.apache.distributedlog.logsegment.LogSegmentEntryReader}.
+ *
+ * NOTE: all the state changes happen in the same thread. All *unsafe* methods should be submitted to the order
+ * scheduler using stream name as the key.
+ */
+public class ReadAheadEntryReader implements
+ AsyncCloseable,
+ LogSegmentListener,
+ LogSegmentEntryReader.StateChangeListener,
+ FutureEventListener<List<Entry.Reader>> {
+
+ private static final Logger logger = LoggerFactory.getLogger(ReadAheadEntryReader.class);
+
+ //
+ // Static Functions
+ //
+
+ private static AbstractFunction1<LogSegmentEntryReader, BoxedUnit> START_READER_FUNC = new AbstractFunction1<LogSegmentEntryReader, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(LogSegmentEntryReader reader) {
+ reader.start();
+ return BoxedUnit.UNIT;
+ }
+ };
+
+ //
+ // Internal Classes
+ //
+
+ class SegmentReader implements FutureEventListener<LogSegmentEntryReader> {
+
+ private LogSegmentMetadata metadata;
+ private final long startEntryId;
+ private Future<LogSegmentEntryReader> openFuture = null;
+ private LogSegmentEntryReader reader = null;
+ private boolean isStarted = false;
+ private boolean isClosed = false;
+
+ SegmentReader(LogSegmentMetadata metadata,
+ long startEntryId) {
+ this.metadata = metadata;
+ this.startEntryId = startEntryId;
+ }
+
+ synchronized LogSegmentEntryReader getEntryReader() {
+ return reader;
+ }
+
+ synchronized boolean isBeyondLastAddConfirmed() {
+ return null != reader && reader.isBeyondLastAddConfirmed();
+ }
+
+ synchronized LogSegmentMetadata getSegment() {
+ return metadata;
+ }
+
+ synchronized boolean isReaderOpen() {
+ return null != openFuture;
+ }
+
+ synchronized void openReader() {
+ if (null != openFuture) {
+ return;
+ }
+ openFuture = entryStore.openReader(metadata, startEntryId).addEventListener(this);
+ }
+
+ synchronized boolean isReaderStarted() {
+ return isStarted;
+ }
+
+ synchronized void startRead() {
+ if (isStarted) {
+ return;
+ }
+ isStarted = true;
+ if (null != reader) {
+ reader.start();
+ } else {
+ openFuture.onSuccess(START_READER_FUNC);
+ }
+ }
+
+ synchronized Future<List<Entry.Reader>> readNext() {
+ if (null != reader) {
+ checkCatchingUpStatus(reader);
+ return reader.readNext(numReadAheadEntries);
+ } else {
+ return openFuture.flatMap(readFunc);
+ }
+ }
+
+ synchronized void updateLogSegmentMetadata(final LogSegmentMetadata segment) {
+ if (null != reader) {
+ reader.onLogSegmentMetadataUpdated(segment);
+ this.metadata = segment;
+ } else {
+ openFuture.onSuccess(new AbstractFunction1<LogSegmentEntryReader, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(LogSegmentEntryReader reader) {
+ reader.onLogSegmentMetadataUpdated(segment);
+ synchronized (SegmentReader.this) {
+ SegmentReader.this.metadata = segment;
+ }
+ return BoxedUnit.UNIT;
+ }
+ });
+ }
+ }
+
+ @Override
+ synchronized public void onSuccess(LogSegmentEntryReader reader) {
+ this.reader = reader;
+ if (reader.getSegment().isInProgress()) {
+ reader.registerListener(ReadAheadEntryReader.this);
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ // no-op, the failure will be propagated on first read.
+ }
+
+ synchronized boolean isClosed() {
+ return isClosed;
+ }
+
+ synchronized Future<Void> close() {
+ if (null == openFuture) {
+ return Future.Void();
+ }
+ return openFuture.flatMap(new AbstractFunction1<LogSegmentEntryReader, Future<Void>>() {
+ @Override
+ public Future<Void> apply(LogSegmentEntryReader reader) {
+ return reader.asyncClose();
+ }
+ }).ensure(new Function0<BoxedUnit>() {
+ @Override
+ public BoxedUnit apply() {
+ synchronized (SegmentReader.this) {
+ isClosed = true;
+ }
+ return null;
+ }
+ });
+ }
+ }
+
+ private class ReadEntriesFunc extends AbstractFunction1<LogSegmentEntryReader, Future<List<Entry.Reader>>> {
+
+ private final int numEntries;
+
+ ReadEntriesFunc(int numEntries) {
+ this.numEntries = numEntries;
+ }
+
+ @Override
+ public Future<List<Entry.Reader>> apply(LogSegmentEntryReader reader) {
+ checkCatchingUpStatus(reader);
+ return reader.readNext(numEntries);
+ }
+ }
+
+ private abstract class CloseableRunnable implements Runnable {
+
+ @Override
+ public void run() {
+ synchronized (ReadAheadEntryReader.this) {
+ if (null != closePromise) {
+ return;
+ }
+ }
+ try {
+ safeRun();
+ } catch (Throwable cause) {
+ logger.error("Caught unexpected exception : ", cause);
+ }
+ }
+
+ abstract void safeRun();
+
+ }
+
+ //
+ // Functions
+ //
+ private final Function1<LogSegmentEntryReader, Future<List<Entry.Reader>>> readFunc;
+ private final Function0<BoxedUnit> removeClosedSegmentReadersFunc = new Function0<BoxedUnit>() {
+ @Override
+ public BoxedUnit apply() {
+ removeClosedSegmentReaders();
+ return BoxedUnit.UNIT;
+ }
+ };
+
+ //
+ // Resources
+ //
+ private final DistributedLogConfiguration conf;
+ private final BKLogReadHandler readHandler;
+ private final LogSegmentEntryStore entryStore;
+ private final OrderedScheduler scheduler;
+
+ //
+ // Parameters
+ //
+ private final String streamName;
+ private final DLSN fromDLSN;
+ private final int maxCachedEntries;
+ private final int numReadAheadEntries;
+ private final int idleWarnThresholdMillis;
+
+ //
+ // Cache
+ //
+ private final LinkedBlockingQueue<Entry.Reader> entryQueue;
+
+ //
+ // State of the reader
+ //
+
+ private final AtomicBoolean started = new AtomicBoolean(false);
+ private boolean isInitialized = false;
+ private boolean readAheadPaused = false;
+ private Promise<Void> closePromise = null;
+ // segment readers
+ private long currentSegmentSequenceNumber;
+ private SegmentReader currentSegmentReader;
+ private SegmentReader nextSegmentReader;
+ private DLSN lastDLSN;
+ private final EntryPosition nextEntryPosition;
+ private volatile boolean isCatchingUp = true;
+ private final LinkedList<SegmentReader> segmentReaders;
+ private final LinkedList<SegmentReader> segmentReadersToClose;
+ // last exception that this reader encounters
+ private final AtomicReference<IOException> lastException = new AtomicReference<IOException>(null);
+ // last entry added time
+ private final Stopwatch lastEntryAddedTime;
+ // state change notification
+ private final CopyOnWriteArraySet<AsyncNotification> stateChangeNotifications =
+ new CopyOnWriteArraySet<AsyncNotification>();
+ // idle reader check task
+ private final ScheduledFuture<?> idleReaderCheckTask;
+
+ //
+ // Stats
+ //
+ private final AlertStatsLogger alertStatsLogger;
+
+ public ReadAheadEntryReader(String streamName,
+ DLSN fromDLSN,
+ DistributedLogConfiguration conf,
+ BKLogReadHandler readHandler,
+ LogSegmentEntryStore entryStore,
+ OrderedScheduler scheduler,
+ Ticker ticker,
+ AlertStatsLogger alertStatsLogger) {
+ this.streamName = streamName;
+ this.fromDLSN = lastDLSN = fromDLSN;
+ this.nextEntryPosition = new EntryPosition(
+ fromDLSN.getLogSegmentSequenceNo(),
+ fromDLSN.getEntryId());
+ this.conf = conf;
+ this.maxCachedEntries = conf.getReadAheadMaxRecords();
+ this.numReadAheadEntries = conf.getReadAheadBatchSize();
+ this.idleWarnThresholdMillis = conf.getReaderIdleWarnThresholdMillis();
+ this.readHandler = readHandler;
+ this.entryStore = entryStore;
+ this.scheduler = scheduler;
+ this.readFunc = new ReadEntriesFunc(numReadAheadEntries);
+ this.alertStatsLogger = alertStatsLogger;
+
+ // create the segment reader list
+ this.segmentReaders = new LinkedList<SegmentReader>();
+ this.segmentReadersToClose = new LinkedList<SegmentReader>();
+ // create the readahead entry queue
+ this.entryQueue = new LinkedBlockingQueue<Entry.Reader>();
+
+ // start the idle reader detection
+ lastEntryAddedTime = Stopwatch.createStarted(ticker);
+ // start the idle reader check task
+ idleReaderCheckTask = scheduleIdleReaderTaskIfNecessary();
+ }
+
+ private ScheduledFuture<?> scheduleIdleReaderTaskIfNecessary() {
+ if (idleWarnThresholdMillis < Integer.MAX_VALUE && idleWarnThresholdMillis > 0) {
+ return scheduler.scheduleAtFixedRate(streamName, new Runnable() {
+ @Override
+ public void run() {
+ if (!isReaderIdle(idleWarnThresholdMillis, TimeUnit.MILLISECONDS)) {
+ return;
+ }
+ // the readahead has been idle
+ unsafeCheckIfReadAheadIsIdle();
+ }
+ }, idleWarnThresholdMillis, idleWarnThresholdMillis, TimeUnit.MILLISECONDS);
+ }
+ return null;
+ }
+
+ private void unsafeCheckIfReadAheadIsIdle() {
+ boolean forceReadLogSegments =
+ (null == currentSegmentReader) || currentSegmentReader.isBeyondLastAddConfirmed();
+ if (forceReadLogSegments) {
+ readHandler.readLogSegmentsFromStore(
+ LogSegmentMetadata.COMPARATOR,
+ LogSegmentFilter.DEFAULT_FILTER,
+ null
+ ).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ // do nothing here since it would be retried on next idle reader check task
+ }
+
+ @Override
+ public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
+ onSegmentsUpdated(segments.getValue());
+ }
+ });
+ }
+ }
+
+ private void cancelIdleReaderTask() {
+ if (null != idleReaderCheckTask) {
+ idleReaderCheckTask.cancel(true);
+ }
+ }
+
+ @VisibleForTesting
+ EntryPosition getNextEntryPosition() {
+ return nextEntryPosition;
+ }
+
+ @VisibleForTesting
+ SegmentReader getCurrentSegmentReader() {
+ return currentSegmentReader;
+ }
+
+ @VisibleForTesting
+ long getCurrentSegmentSequenceNumber() {
+ return currentSegmentSequenceNumber;
+ }
+
+ @VisibleForTesting
+ SegmentReader getNextSegmentReader() {
+ return nextSegmentReader;
+ }
+
+ @VisibleForTesting
+ LinkedList<SegmentReader> getSegmentReaders() {
+ return segmentReaders;
+ }
+
+ @VisibleForTesting
+ boolean isInitialized() {
+ return isInitialized;
+ }
+
+ private void orderedSubmit(Runnable runnable) {
+ synchronized (this) {
+ if (null != closePromise) {
+ return;
+ }
+ }
+ try {
+ scheduler.submit(streamName, runnable);
+ } catch (RejectedExecutionException ree) {
+ logger.debug("Failed to submit and execute an operation for readhead entry reader of {}",
+ streamName, ree);
+ }
+ }
+
+ public void start(final List<LogSegmentMetadata> segmentList) {
+ logger.info("Starting the readahead entry reader for {} : segments = {}",
+ readHandler.getFullyQualifiedName(), segmentList);
+ started.set(true);
+ processLogSegments(segmentList);
+ }
+
+ private void removeClosedSegmentReaders() {
+ orderedSubmit(new CloseableRunnable() {
+ @Override
+ void safeRun() {
+ unsafeRemoveClosedSegmentReaders();
+ }
+ });
+ }
+
+ private void unsafeRemoveClosedSegmentReaders() {
+ SegmentReader reader = segmentReadersToClose.peekFirst();
+ while (null != reader) {
+ if (reader.isClosed()) {
+ segmentReadersToClose.pollFirst();
+ reader = segmentReadersToClose.peekFirst();
+ } else {
+ break;
+ }
+ }
+ }
+
+ @Override
+ public Future<Void> asyncClose() {
+ final Promise<Void> closeFuture;
+ synchronized (this) {
+ if (null != closePromise) {
+ return closePromise;
+ }
+ closePromise = closeFuture = new Promise<Void>();
+ }
+
+ // cancel the idle reader task
+ cancelIdleReaderTask();
+
+ // use runnable here instead of CloseableRunnable,
+ // because we need this to be executed
+ try {
+ scheduler.submit(streamName, new Runnable() {
+ @Override
+ public void run() {
+ unsafeAsyncClose(closeFuture);
+ }
+ });
+ } catch (RejectedExecutionException ree) {
+ logger.warn("Scheduler has been shutdown before closing the readahead entry reader for stream {}",
+ streamName, ree);
+ unsafeAsyncClose(closeFuture);
+ }
+
+ return closeFuture;
+ }
+
+ private void unsafeAsyncClose(Promise<Void> closePromise) {
+ List<Future<Void>> closeFutures = Lists.newArrayListWithExpectedSize(
+ segmentReaders.size() + segmentReadersToClose.size() + 1);
+ if (null != currentSegmentReader) {
+ segmentReadersToClose.add(currentSegmentReader);
+ }
+ if (null != nextSegmentReader) {
+ segmentReadersToClose.add(nextSegmentReader);
+ }
+ for (SegmentReader reader : segmentReaders) {
+ segmentReadersToClose.add(reader);
+ }
+ segmentReaders.clear();
+ for (SegmentReader reader : segmentReadersToClose) {
+ closeFutures.add(reader.close());
+ }
+ Futures.collect(closeFutures).proxyTo(closePromise);
+ }
+
+ //
+ // Reader State Changes
+ //
+
+ ReadAheadEntryReader addStateChangeNotification(AsyncNotification notification) {
+ this.stateChangeNotifications.add(notification);
+ return this;
+ }
+
+ ReadAheadEntryReader removeStateChangeNotification(AsyncNotification notification) {
+ this.stateChangeNotifications.remove(notification);
+ return this;
+ }
+
+ private void notifyStateChangeOnSuccess() {
+ for (AsyncNotification notification : stateChangeNotifications) {
+ notification.notifyOnOperationComplete();
+ }
+ }
+
+ private void notifyStateChangeOnFailure(Throwable cause) {
+ for (AsyncNotification notification : stateChangeNotifications) {
+ notification.notifyOnError(cause);
+ }
+ }
+
+ void setLastException(IOException cause) {
+ if (!lastException.compareAndSet(null, cause)) {
+ logger.debug("last exception has already been set to ", lastException.get());
+ }
+ // the exception is set and notify the state change
+ notifyStateChangeOnFailure(cause);
+ }
+
+ void checkLastException() throws IOException {
+ if (null != lastException.get()) {
+ throw lastException.get();
+ }
+ }
+
+ void checkCatchingUpStatus(LogSegmentEntryReader reader) {
+ if (reader.getSegment().isInProgress()
+ && isCatchingUp
+ && reader.hasCaughtUpOnInprogress()) {
+ logger.info("ReadAhead for {} is caught up at entry {} @ log segment {}.",
+ new Object[] { readHandler.getFullyQualifiedName(),
+ reader.getLastAddConfirmed(), reader.getSegment() });
+ isCatchingUp = false;
+ }
+ }
+
+ 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
+ //
+
+ @Override
+ public void onSuccess(List<Entry.Reader> entries) {
+ lastEntryAddedTime.reset().start();
+ for (Entry.Reader entry : entries) {
+ entryQueue.add(entry);
+ }
+ if (!entries.isEmpty()) {
+ Entry.Reader lastEntry = entries.get(entries.size() - 1);
+ nextEntryPosition.advance(lastEntry.getLSSN(), lastEntry.getEntryId() + 1);
+ }
+ // notify on data available
+ notifyStateChangeOnSuccess();
+ if (entryQueue.size() >= maxCachedEntries) {
+ pauseReadAheadOnCacheFull();
+ } else {
+ scheduleReadNext();
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ if (cause instanceof EndOfLogSegmentException) {
+ // we reach end of the log segment
+ moveToNextLogSegment();
+ return;
+ }
+ if (cause instanceof IOException) {
+ setLastException((IOException) cause);
+ } else {
+ setLastException(new UnexpectedException("Unexpected non I/O exception", cause));
+ }
+ }
+
+ private synchronized void invokeReadAhead() {
+ if (readAheadPaused) {
+ scheduleReadNext();
+ readAheadPaused = false;
+ }
+ }
+
+ private synchronized void pauseReadAheadOnCacheFull() {
+ this.readAheadPaused = true;
+ if (!isCacheFull()) {
+ invokeReadAhead();
+ }
+ }
+
+ private synchronized void pauseReadAheadOnNoMoreLogSegments() {
+ this.readAheadPaused = true;
+ }
+
+ //
+ // Cache Related Methods
+ //
+
+ public Entry.Reader getNextReadAheadEntry(long waitTime, TimeUnit waitTimeUnit) throws IOException {
+ if (null != lastException.get()) {
+ throw lastException.get();
+ }
+ Entry.Reader entry;
+ try {
+ entry = entryQueue.poll(waitTime, waitTimeUnit);
+ } catch (InterruptedException e) {
+ throw new DLInterruptedException("Interrupted on waiting next readahead entry : ", e);
+ }
+ try {
+ return entry;
+ } finally {
+ // resume readahead if the cache becomes empty
+ if (null != entry && !isCacheFull()) {
+ invokeReadAhead();
+ }
+ }
+ }
+
+ /**
+ * Return number cached entries.
+ *
+ * @return number cached entries.
+ */
+ public int getNumCachedEntries() {
+ return entryQueue.size();
+ }
+
+ /**
+ * Return if the cache is full.
+ *
+ * @return true if the cache is full, otherwise false.
+ */
+ public boolean isCacheFull() {
+ return getNumCachedEntries() >= maxCachedEntries;
+ }
+
+ @VisibleForTesting
+ public boolean isCacheEmpty() {
+ return entryQueue.isEmpty();
+ }
+
+ /**
+ * Check whether the readahead becomes stall.
+ *
+ * @param idleReaderErrorThreshold idle reader error threshold
+ * @param timeUnit time unit of the idle reader error threshold
+ * @return true if the readahead becomes stall, otherwise false.
+ */
+ public boolean isReaderIdle(int idleReaderErrorThreshold, TimeUnit timeUnit) {
+ return (lastEntryAddedTime.elapsed(timeUnit) > idleReaderErrorThreshold);
+ }
+
+ //
+ // LogSegment Management
+ //
+
+ void processLogSegments(final List<LogSegmentMetadata> segments) {
+ orderedSubmit(new CloseableRunnable() {
+ @Override
+ void safeRun() {
+ unsafeProcessLogSegments(segments);
+ }
+ });
+ }
+
+ private void unsafeProcessLogSegments(List<LogSegmentMetadata> segments) {
+ if (isInitialized) {
+ unsafeReinitializeLogSegments(segments);
+ } else {
+ unsafeInitializeLogSegments(segments);
+ }
+ }
+
+ /**
+ * Update the log segment metadata.
+ *
+ * @param reader the reader to update the metadata
+ * @param newMetadata the new metadata received
+ * @return true if successfully, false on encountering errors
+ */
+ private boolean updateLogSegmentMetadata(SegmentReader reader,
+ LogSegmentMetadata newMetadata) {
+ if (reader.getSegment().getLogSegmentSequenceNumber() != newMetadata.getLogSegmentSequenceNumber()) {
+ setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for "
+ + streamName + " : current segment = " + reader.getSegment() + ", new segment = " + newMetadata));
+ return false;
+ }
+ if (!reader.getSegment().isInProgress() && newMetadata.isInProgress()) {
+ setLastException(new DLIllegalStateException("An inprogress log segment " + newMetadata
+ + " received after a closed log segment " + reader.getSegment() + " on reading segment "
+ + newMetadata.getLogSegmentSequenceNumber() + " @ stream " + streamName));
+ return false;
+ }
+ if (reader.getSegment().isInProgress() && !newMetadata.isInProgress()) {
+ reader.updateLogSegmentMetadata(newMetadata);
+ }
+ return true;
+ }
+
+ /**
+ * Reinitialize the log segments
+ */
+ private void unsafeReinitializeLogSegments(List<LogSegmentMetadata> segments) {
+ logger.info("Reinitialize log segments with {}", segments);
+ int segmentIdx = 0;
+ for (; segmentIdx < segments.size(); segmentIdx++) {
+ LogSegmentMetadata segment = segments.get(segmentIdx);
+ if (segment.getLogSegmentSequenceNumber() < currentSegmentSequenceNumber) {
+ continue;
+ }
+ break;
+ }
+ if (segmentIdx >= segments.size()) {
+ return;
+ }
+ LogSegmentMetadata segment = segments.get(segmentIdx);
+ if (null != currentSegmentReader) {
+ if (!updateLogSegmentMetadata(currentSegmentReader, segment)) {
+ return;
+ }
+ } else {
+ if (currentSegmentSequenceNumber != segment.getLogSegmentSequenceNumber()) {
+ setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for "
+ + streamName + " : current segment sn = " + currentSegmentSequenceNumber
+ + ", new segment sn = " + segment.getLogSegmentSequenceNumber()));
+ return;
+ }
+ }
+ segmentIdx++;
+ if (segmentIdx >= segments.size()) {
+ return;
+ }
+ // check next segment
+ segment = segments.get(segmentIdx);
+ if (null != nextSegmentReader) {
+ if (!updateLogSegmentMetadata(nextSegmentReader, segment)) {
+ return;
+ }
+ segmentIdx++;
+ }
+ // check the segment readers in the queue
+ for (int readerIdx = 0;
+ readerIdx < segmentReaders.size() && segmentIdx < segments.size();
+ readerIdx++, segmentIdx++) {
+ SegmentReader reader = segmentReaders.get(readerIdx);
+ segment = segments.get(segmentIdx);
+ if (!updateLogSegmentMetadata(reader, segment)) {
+ return;
+ }
+ }
+ // add the remaining segments to the reader queue
+ for (; segmentIdx < segments.size(); segmentIdx++) {
+ segment = segments.get(segmentIdx);
+ SegmentReader reader = new SegmentReader(segment, 0L);
+ reader.openReader();
+ segmentReaders.add(reader);
+ }
+ if (null == currentSegmentReader) {
+ unsafeMoveToNextLogSegment();
+ }
+ // resume readahead if necessary
+ invokeReadAhead();
+ }
+
+ /**
+ * Initialize the reader with the log <i>segments</i>.
+ *
+ * @param segments list of log segments
+ */
+ private void unsafeInitializeLogSegments(List<LogSegmentMetadata> segments) {
+ if (segments.isEmpty()) {
+ // not initialize the background reader, until the first log segment is notified
+ return;
+ }
+ boolean skipTruncatedLogSegments = true;
+ DLSN dlsnToStart = fromDLSN;
+ // positioning the reader
+ for (int i = 0; i < segments.size(); i++) {
+ LogSegmentMetadata segment = segments.get(i);
+ // skip any log segments that have smaller log segment sequence numbers
+ if (segment.getLogSegmentSequenceNumber() < fromDLSN.getLogSegmentSequenceNo()) {
+ continue;
+ }
+ // if the log segment is truncated, skip it.
+ if (skipTruncatedLogSegments &&
+ !conf.getIgnoreTruncationStatus() &&
+ segment.isTruncated()) {
+ continue;
+ }
+ // if the log segment is partially truncated, move the start dlsn to the min active dlsn
+ if (skipTruncatedLogSegments &&
+ !conf.getIgnoreTruncationStatus() &&
+ segment.isPartiallyTruncated()) {
+ if (segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
+ dlsnToStart = segment.getMinActiveDLSN();
+ }
+ }
+ skipTruncatedLogSegments = false;
+ if (!isAllowedToPosition(segment, dlsnToStart)) {
+ logger.error("segment {} is not allowed to position at {}", segment, dlsnToStart);
+ return;
+ }
+
+ SegmentReader reader = new SegmentReader(segment,
+ segment.getLogSegmentSequenceNumber() == dlsnToStart.getLogSegmentSequenceNo()
+ ? dlsnToStart.getEntryId() : 0L);
+ segmentReaders.add(reader);
+ }
+ if (segmentReaders.isEmpty()) {
+ // not initialize the background reader, until the first log segment is available to read
+ return;
+ }
+ currentSegmentReader = segmentReaders.pollFirst();
+ currentSegmentReader.openReader();
+ currentSegmentReader.startRead();
+ currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
+ unsafeReadNext(currentSegmentReader);
+ if (!segmentReaders.isEmpty()) {
+ for (SegmentReader reader : segmentReaders) {
+ reader.openReader();
+ }
+ unsafePrefetchNextSegment(true);
+ }
+ // mark the reader initialized
+ isInitialized = true;
+ }
+
+ private void unsafePrefetchNextSegment(boolean onlyInprogressLogSegment) {
+ SegmentReader nextReader = segmentReaders.peekFirst();
+ // open the next log segment if it is inprogress
+ if (null != nextReader) {
+ if (onlyInprogressLogSegment && !nextReader.getSegment().isInProgress()) {
+ return;
+ }
+ nextReader.startRead();
+ nextSegmentReader = nextReader;
+ segmentReaders.pollFirst();
+ }
+ }
+
+ /**
+ * Check if we are allowed to position the reader at <i>fromDLSN</i>.
+ *
+ * @return true if it is allowed, otherwise false.
+ */
+ private boolean isAllowedToPosition(LogSegmentMetadata segment, DLSN fromDLSN) {
+ if (segment.isTruncated()
+ && segment.getLastDLSN().compareTo(fromDLSN) >= 0
+ && !conf.getIgnoreTruncationStatus()) {
+ setLastException(new AlreadyTruncatedTransactionException(streamName
+ + " : trying to position read ahead at " + fromDLSN
+ + " on a segment " + segment + " that is already marked as truncated"));
+ return false;
+ }
+ if (segment.isPartiallyTruncated() &&
+ segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
+ if (conf.getAlertWhenPositioningOnTruncated()) {
+ alertStatsLogger.raise("Trying to position reader on {} when {} is marked partially truncated",
+ fromDLSN, segment);
+ }
+ if (!conf.getIgnoreTruncationStatus()) {
+ logger.error("{}: Trying to position reader on {} when {} is marked partially truncated",
+ new Object[]{ streamName, fromDLSN, segment });
+
+ setLastException(new AlreadyTruncatedTransactionException(streamName
+ + " : trying to position read ahead at " + fromDLSN
+ + " on a segment " + segment + " that is already marked as truncated"));
+ return false;
+ }
+ }
+ return true;
+ }
+
+ void moveToNextLogSegment() {
+ orderedSubmit(new CloseableRunnable() {
+ @Override
+ void safeRun() {
+ unsafeMoveToNextLogSegment();
+ }
+ });
+ }
+
+ private void unsafeMoveToNextLogSegment() {
+ if (null != currentSegmentReader) {
+ segmentReadersToClose.add(currentSegmentReader);
+ currentSegmentReader.close().ensure(removeClosedSegmentReadersFunc);
+ logger.debug("close current segment reader {}", currentSegmentReader.getSegment());
+ currentSegmentReader = null;
+ }
+ boolean hasSegmentToRead = false;
+ if (null != nextSegmentReader) {
+ currentSegmentReader = nextSegmentReader;
+ logger.debug("move to read segment {}", currentSegmentReader.getSegment());
+ currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
+ nextSegmentReader = null;
+ // start reading
+ unsafeReadNext(currentSegmentReader);
+ unsafePrefetchNextSegment(true);
+ hasSegmentToRead = true;
+ } else {
+ unsafePrefetchNextSegment(false);
+ if (null != nextSegmentReader) {
+ currentSegmentReader = nextSegmentReader;
+ logger.debug("move to read segment {}", currentSegmentReader.getSegment());
+ currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
+ nextSegmentReader = null;
+ unsafeReadNext(currentSegmentReader);
+ unsafePrefetchNextSegment(true);
+ hasSegmentToRead = true;
+ }
+ }
+ if (!hasSegmentToRead) { // no more segment to read, wait until new log segment arrive
+ if (isCatchingUp) {
+ logger.info("ReadAhead for {} is caught up and no log segments to read now",
+ readHandler.getFullyQualifiedName());
+ isCatchingUp = false;
+ }
+ pauseReadAheadOnNoMoreLogSegments();
+ }
+ }
+
+ void scheduleReadNext() {
+ orderedSubmit(new CloseableRunnable() {
+ @Override
+ void safeRun() {
+ if (null == currentSegmentReader) {
+ pauseReadAheadOnNoMoreLogSegments();
+ return;
+ }
+ unsafeReadNext(currentSegmentReader);
+ }
+ });
+ }
+
+ private void unsafeReadNext(SegmentReader reader) {
+ reader.readNext().addEventListener(this);
+ }
+
+ @Override
+ public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
+ if (!started.get()) {
+ return;
+ }
+ logger.info("segments is updated with {}", segments);
+ processLogSegments(segments);
+ }
+
+ @Override
+ public void onLogStreamDeleted() {
+ setLastException(new LogNotFoundException("Log stream "
+ + streamName + " is deleted"));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
new file mode 100644
index 0000000..9935d5f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
@@ -0,0 +1,782 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.selector.FirstDLSNNotLessThanSelector;
+import org.apache.distributedlog.selector.FirstTxIdNotLessThanSelector;
+import org.apache.distributedlog.selector.LastRecordSelector;
+import org.apache.distributedlog.selector.LogRecordSelector;
+import org.apache.distributedlog.util.FutureUtils.FutureEventListenerRunnable;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction0;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Utility function for readers
+ */
+public class ReadUtils {
+
+ static final Logger LOG = LoggerFactory.getLogger(ReadUtils.class);
+
+ private static final int MIN_SEARCH_BATCH_SIZE = 2;
+
+ //
+ // Read First & Last Record Functions
+ //
+
+ /**
+ * Read last record from a log segment.
+ *
+ * @param streamName
+ * fully qualified stream name (used for logging)
+ * @param l
+ * log segment metadata.
+ * @param fence
+ * whether to fence the log segment.
+ * @param includeControl
+ * whether to include control record.
+ * @param includeEndOfStream
+ * whether to include end of stream.
+ * @param scanStartBatchSize
+ * first num entries used for read last record scan
+ * @param scanMaxBatchSize
+ * max num entries used for read last record scan
+ * @param numRecordsScanned
+ * num of records scanned to get last record
+ * @param executorService
+ * executor service used for processing entries
+ * @param entryStore
+ * log segment entry store
+ * @return a future with last record.
+ */
+ public static Future<LogRecordWithDLSN> asyncReadLastRecord(
+ final String streamName,
+ final LogSegmentMetadata l,
+ final boolean fence,
+ final boolean includeControl,
+ final boolean includeEndOfStream,
+ final int scanStartBatchSize,
+ final int scanMaxBatchSize,
+ final AtomicInteger numRecordsScanned,
+ final ExecutorService executorService,
+ final LogSegmentEntryStore entryStore) {
+ final LogRecordSelector selector = new LastRecordSelector();
+ return asyncReadRecord(streamName, l, fence, includeControl, includeEndOfStream, scanStartBatchSize,
+ scanMaxBatchSize, numRecordsScanned, executorService, entryStore,
+ selector, true /* backward */, 0L);
+ }
+
+ /**
+ * Read first record from a log segment with a DLSN larger than that given.
+ *
+ * @param streamName
+ * fully qualified stream name (used for logging)
+ * @param l
+ * log segment metadata.
+ * @param scanStartBatchSize
+ * first num entries used for read last record scan
+ * @param scanMaxBatchSize
+ * max num entries used for read last record scan
+ * @param numRecordsScanned
+ * num of records scanned to get last record
+ * @param executorService
+ * executor service used for processing entries
+ * @param entryStore
+ * log segment entry store
+ * @param dlsn
+ * threshold dlsn
+ * @return a future with last record.
+ */
+ public static Future<LogRecordWithDLSN> asyncReadFirstUserRecord(
+ final String streamName,
+ final LogSegmentMetadata l,
+ final int scanStartBatchSize,
+ final int scanMaxBatchSize,
+ final AtomicInteger numRecordsScanned,
+ final ExecutorService executorService,
+ final LogSegmentEntryStore entryStore,
+ final DLSN dlsn) {
+ long startEntryId = 0L;
+ if (l.getLogSegmentSequenceNumber() == dlsn.getLogSegmentSequenceNo()) {
+ startEntryId = dlsn.getEntryId();
+ }
+ final LogRecordSelector selector = new FirstDLSNNotLessThanSelector(dlsn);
+ return asyncReadRecord(streamName, l, false, false, false, scanStartBatchSize,
+ scanMaxBatchSize, numRecordsScanned, executorService, entryStore,
+ selector, false /* backward */, startEntryId);
+ }
+
+ //
+ // Private methods for scanning log segments
+ //
+
+ private static class ScanContext {
+ // variables to about current scan state
+ final AtomicInteger numEntriesToScan;
+ final AtomicLong curStartEntryId;
+ final AtomicLong curEndEntryId;
+
+ // scan settings
+ final long startEntryId;
+ final long endEntryId;
+ final int scanStartBatchSize;
+ final int scanMaxBatchSize;
+ final boolean includeControl;
+ final boolean includeEndOfStream;
+ final boolean backward;
+
+ // number of records scanned
+ final AtomicInteger numRecordsScanned;
+
+ ScanContext(long startEntryId, long endEntryId,
+ int scanStartBatchSize,
+ int scanMaxBatchSize,
+ boolean includeControl,
+ boolean includeEndOfStream,
+ boolean backward,
+ AtomicInteger numRecordsScanned) {
+ this.startEntryId = startEntryId;
+ this.endEntryId = endEntryId;
+ this.scanStartBatchSize = scanStartBatchSize;
+ this.scanMaxBatchSize = scanMaxBatchSize;
+ this.includeControl = includeControl;
+ this.includeEndOfStream = includeEndOfStream;
+ this.backward = backward;
+ // Scan state
+ this.numEntriesToScan = new AtomicInteger(scanStartBatchSize);
+ if (backward) {
+ this.curStartEntryId = new AtomicLong(
+ Math.max(startEntryId, (endEntryId - scanStartBatchSize + 1)));
+ this.curEndEntryId = new AtomicLong(endEntryId);
+ } else {
+ this.curStartEntryId = new AtomicLong(startEntryId);
+ this.curEndEntryId = new AtomicLong(
+ Math.min(endEntryId, (startEntryId + scanStartBatchSize - 1)));
+ }
+ this.numRecordsScanned = numRecordsScanned;
+ }
+
+ boolean moveToNextRange() {
+ if (backward) {
+ return moveBackward();
+ } else {
+ return moveForward();
+ }
+ }
+
+ boolean moveBackward() {
+ long nextEndEntryId = curStartEntryId.get() - 1;
+ if (nextEndEntryId < startEntryId) {
+ // no entries to read again
+ return false;
+ }
+ curEndEntryId.set(nextEndEntryId);
+ // update num entries to scan
+ numEntriesToScan.set(
+ Math.min(numEntriesToScan.get() * 2, scanMaxBatchSize));
+ // update start entry id
+ curStartEntryId.set(Math.max(startEntryId, nextEndEntryId - numEntriesToScan.get() + 1));
+ return true;
+ }
+
+ boolean moveForward() {
+ long nextStartEntryId = curEndEntryId.get() + 1;
+ if (nextStartEntryId > endEntryId) {
+ // no entries to read again
+ return false;
+ }
+ curStartEntryId.set(nextStartEntryId);
+ // update num entries to scan
+ numEntriesToScan.set(
+ Math.min(numEntriesToScan.get() * 2, scanMaxBatchSize));
+ // update start entry id
+ curEndEntryId.set(Math.min(endEntryId, nextStartEntryId + numEntriesToScan.get() - 1));
+ return true;
+ }
+ }
+
+ private static class SingleEntryScanContext extends ScanContext {
+ SingleEntryScanContext(long entryId) {
+ super(entryId, entryId, 1, 1, true, true, false, new AtomicInteger(0));
+ }
+ }
+
+ /**
+ * Read record from a given range of log segment entries.
+ *
+ * @param streamName
+ * fully qualified stream name (used for logging)
+ * @param reader
+ * log segment random access reader
+ * @param executorService
+ * executor service used for processing entries
+ * @param context
+ * scan context
+ * @return a future with the log record.
+ */
+ private static Future<LogRecordWithDLSN> asyncReadRecordFromEntries(
+ final String streamName,
+ final LogSegmentRandomAccessEntryReader reader,
+ final LogSegmentMetadata metadata,
+ final ExecutorService executorService,
+ final ScanContext context,
+ final LogRecordSelector selector) {
+ final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
+ final long startEntryId = context.curStartEntryId.get();
+ final long endEntryId = context.curEndEntryId.get();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{} reading entries [{} - {}] from {}.",
+ new Object[] { streamName, startEntryId, endEntryId, metadata});
+ }
+ FutureEventListener<List<Entry.Reader>> readEntriesListener =
+ new FutureEventListener<List<Entry.Reader>>() {
+ @Override
+ public void onSuccess(final List<Entry.Reader> entries) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{} finished reading entries [{} - {}] from {}",
+ new Object[]{ streamName, startEntryId, endEntryId, metadata});
+ }
+ for (Entry.Reader entry : entries) {
+ try {
+ visitEntryRecords(entry, context, selector);
+ } catch (IOException ioe) {
+ // exception is only thrown due to bad ledger entry, so it might be corrupted
+ // we shouldn't do anything beyond this point. throw the exception to application
+ promise.setException(ioe);
+ return;
+ }
+ }
+
+ LogRecordWithDLSN record = selector.result();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{} got record from entries [{} - {}] of {} : {}",
+ new Object[]{streamName, startEntryId, endEntryId,
+ metadata, record});
+ }
+ promise.setValue(record);
+ }
+
+ @Override
+ public void onFailure(final Throwable cause) {
+ promise.setException(cause);
+ }
+ };
+ reader.readEntries(startEntryId, endEntryId)
+ .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
+ return promise;
+ }
+
+ /**
+ * Process each record using LogRecordSelector.
+ *
+ * @param entry
+ * ledger entry
+ * @param context
+ * scan context
+ * @return log record with dlsn inside the ledger entry
+ * @throws IOException
+ */
+ private static void visitEntryRecords(
+ Entry.Reader entry,
+ ScanContext context,
+ LogRecordSelector selector) throws IOException {
+ LogRecordWithDLSN nextRecord = entry.nextRecord();
+ while (nextRecord != null) {
+ LogRecordWithDLSN record = nextRecord;
+ nextRecord = entry.nextRecord();
+ context.numRecordsScanned.incrementAndGet();
+ if (!context.includeControl && record.isControl()) {
+ continue;
+ }
+ if (!context.includeEndOfStream && record.isEndOfStream()) {
+ continue;
+ }
+ selector.process(record);
+ }
+ }
+
+ /**
+ * Scan entries for the given record.
+ *
+ * @param streamName
+ * fully qualified stream name (used for logging)
+ * @param reader
+ * log segment random access reader
+ * @param executorService
+ * executor service used for processing entries
+ * @param promise
+ * promise to return desired record.
+ * @param context
+ * scan context
+ */
+ private static void asyncReadRecordFromEntries(
+ final String streamName,
+ final LogSegmentRandomAccessEntryReader reader,
+ final LogSegmentMetadata metadata,
+ final ExecutorService executorService,
+ final Promise<LogRecordWithDLSN> promise,
+ final ScanContext context,
+ final LogRecordSelector selector) {
+ FutureEventListener<LogRecordWithDLSN> readEntriesListener =
+ new FutureEventListener<LogRecordWithDLSN>() {
+ @Override
+ public void onSuccess(LogRecordWithDLSN value) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{} read record from [{} - {}] of {} : {}",
+ new Object[]{streamName, context.curStartEntryId.get(), context.curEndEntryId.get(),
+ metadata, value});
+ }
+ if (null != value) {
+ promise.setValue(value);
+ return;
+ }
+ if (!context.moveToNextRange()) {
+ // no entries to read again
+ promise.setValue(null);
+ return;
+ }
+ // scan next range
+ asyncReadRecordFromEntries(streamName,
+ reader,
+ metadata,
+ executorService,
+ promise,
+ context,
+ selector);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ promise.setException(cause);
+ }
+ };
+ asyncReadRecordFromEntries(streamName, reader, metadata, executorService, context, selector)
+ .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
+ }
+
+ private static void asyncReadRecordFromLogSegment(
+ final String streamName,
+ final LogSegmentRandomAccessEntryReader reader,
+ final LogSegmentMetadata metadata,
+ final ExecutorService executorService,
+ final int scanStartBatchSize,
+ final int scanMaxBatchSize,
+ final boolean includeControl,
+ final boolean includeEndOfStream,
+ final Promise<LogRecordWithDLSN> promise,
+ final AtomicInteger numRecordsScanned,
+ final LogRecordSelector selector,
+ final boolean backward,
+ final long startEntryId) {
+ final long lastAddConfirmed = reader.getLastAddConfirmed();
+ if (lastAddConfirmed < 0) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Log segment {} is empty for {}.", new Object[] { metadata, streamName });
+ }
+ promise.setValue(null);
+ return;
+ }
+ final ScanContext context = new ScanContext(
+ startEntryId, lastAddConfirmed,
+ scanStartBatchSize, scanMaxBatchSize,
+ includeControl, includeEndOfStream, backward, numRecordsScanned);
+ asyncReadRecordFromEntries(streamName, reader, metadata, executorService,
+ promise, context, selector);
+ }
+
+ private static Future<LogRecordWithDLSN> asyncReadRecord(
+ final String streamName,
+ final LogSegmentMetadata l,
+ final boolean fence,
+ final boolean includeControl,
+ final boolean includeEndOfStream,
+ final int scanStartBatchSize,
+ final int scanMaxBatchSize,
+ final AtomicInteger numRecordsScanned,
+ final ExecutorService executorService,
+ final LogSegmentEntryStore entryStore,
+ final LogRecordSelector selector,
+ final boolean backward,
+ final long startEntryId) {
+
+ final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
+
+ FutureEventListener<LogSegmentRandomAccessEntryReader> openReaderListener =
+ new FutureEventListener<LogSegmentRandomAccessEntryReader>() {
+ @Override
+ public void onSuccess(final LogSegmentRandomAccessEntryReader reader) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{} Opened log segment {} for reading record",
+ streamName, l);
+ }
+ promise.ensure(new AbstractFunction0<BoxedUnit>() {
+ @Override
+ public BoxedUnit apply() {
+ reader.asyncClose();
+ return BoxedUnit.UNIT;
+ }
+ });
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("{} {} scanning {}.", new Object[]{
+ (backward ? "backward" : "forward"), streamName, l});
+ }
+ asyncReadRecordFromLogSegment(
+ streamName, reader, l, executorService,
+ scanStartBatchSize, scanMaxBatchSize,
+ includeControl, includeEndOfStream,
+ promise, numRecordsScanned, selector, backward, startEntryId);
+ }
+
+ @Override
+ public void onFailure(final Throwable cause) {
+ promise.setException(cause);
+ }
+ };
+ entryStore.openRandomAccessReader(l, fence)
+ .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
+ return promise;
+ }
+
+ //
+ // Search Functions
+ //
+
+ /**
+ * Get the log record whose transaction id is not less than provided <code>transactionId</code>.
+ *
+ * <p>
+ * It uses a binary-search like algorithm to find the log record whose transaction id is not less than
+ * provided <code>transactionId</code> within a log <code>segment</code>. You could think of a log segment
+ * in terms of a sequence of records whose transaction ids are non-decreasing.
+ *
+ * - The sequence of records within a log segment is divided into N pieces.
+ * - Find the piece of records that contains a record whose transaction id is not less than provided
+ * <code>transactionId</code>.
+ *
+ * N could be chosen based on trading off concurrency and latency.
+ * </p>
+ *
+ * @param logName
+ * name of the log
+ * @param segment
+ * metadata of the log segment
+ * @param transactionId
+ * transaction id
+ * @param executorService
+ * executor service used for processing entries
+ * @param entryStore
+ * log segment entry store
+ * @param nWays
+ * how many number of entries to search in parallel
+ * @return found log record. none if all transaction ids are less than provided <code>transactionId</code>.
+ */
+ public static Future<Optional<LogRecordWithDLSN>> getLogRecordNotLessThanTxId(
+ final String logName,
+ final LogSegmentMetadata segment,
+ final long transactionId,
+ final ExecutorService executorService,
+ final LogSegmentEntryStore entryStore,
+ final int nWays) {
+ if (!segment.isInProgress()) {
+ if (segment.getLastTxId() < transactionId) {
+ // all log records whose transaction id is less than provided transactionId
+ // then return none
+ Optional<LogRecordWithDLSN> noneRecord = Optional.absent();
+ return Future.value(noneRecord);
+ }
+ }
+
+ final Promise<Optional<LogRecordWithDLSN>> promise =
+ new Promise<Optional<LogRecordWithDLSN>>();
+ final FutureEventListener<LogSegmentRandomAccessEntryReader> openReaderListener =
+ new FutureEventListener<LogSegmentRandomAccessEntryReader>() {
+ @Override
+ public void onSuccess(final LogSegmentRandomAccessEntryReader reader) {
+ promise.ensure(new AbstractFunction0<BoxedUnit>() {
+ @Override
+ public BoxedUnit apply() {
+ reader.asyncClose();
+ return BoxedUnit.UNIT;
+ }
+
+ });
+ long lastEntryId = reader.getLastAddConfirmed();
+ if (lastEntryId < 0) {
+ // it means that the log segment is created but not written yet or an empty log segment.
+ // it is equivalent to 'all log records whose transaction id is less than provided transactionId'
+ Optional<LogRecordWithDLSN> nonRecord = Optional.absent();
+ promise.setValue(nonRecord);
+ return;
+ }
+ // all log records whose transaction id is not less than provided transactionId
+ if (segment.getFirstTxId() >= transactionId) {
+ final FirstTxIdNotLessThanSelector selector =
+ new FirstTxIdNotLessThanSelector(transactionId);
+ asyncReadRecordFromEntries(
+ logName,
+ reader,
+ segment,
+ executorService,
+ new SingleEntryScanContext(0L),
+ selector
+ ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+ @Override
+ public void onSuccess(LogRecordWithDLSN value) {
+ promise.setValue(Optional.of(selector.result()));
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ promise.setException(cause);
+ }
+ });
+
+ return;
+ }
+ getLogRecordNotLessThanTxIdFromEntries(
+ logName,
+ segment,
+ transactionId,
+ executorService,
+ reader,
+ Lists.newArrayList(0L, lastEntryId),
+ nWays,
+ Optional.<LogRecordWithDLSN>absent(),
+ promise);
+ }
+
+ @Override
+ public void onFailure(final Throwable cause) {
+ promise.setException(cause);
+ }
+ };
+
+ entryStore.openRandomAccessReader(segment, false)
+ .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
+ return promise;
+ }
+
+ /**
+ * Find the log record whose transaction id is not less than provided <code>transactionId</code> from
+ * entries between <code>startEntryId</code> and <code>endEntryId</code>.
+ *
+ * @param logName
+ * name of the log
+ * @param segment
+ * log segment
+ * @param transactionId
+ * provided transaction id to search
+ * @param executorService
+ * executor service
+ * @param reader
+ * log segment random access reader
+ * @param entriesToSearch
+ * list of entries to search
+ * @param nWays
+ * how many entries to search in parallel
+ * @param prevFoundRecord
+ * the log record found in previous search
+ * @param promise
+ * promise to satisfy the result
+ */
+ private static void getLogRecordNotLessThanTxIdFromEntries(
+ final String logName,
+ final LogSegmentMetadata segment,
+ final long transactionId,
+ final ExecutorService executorService,
+ final LogSegmentRandomAccessEntryReader reader,
+ final List<Long> entriesToSearch,
+ final int nWays,
+ final Optional<LogRecordWithDLSN> prevFoundRecord,
+ final Promise<Optional<LogRecordWithDLSN>> promise) {
+ final List<Future<LogRecordWithDLSN>> searchResults =
+ Lists.newArrayListWithExpectedSize(entriesToSearch.size());
+ for (Long entryId : entriesToSearch) {
+ LogRecordSelector selector = new FirstTxIdNotLessThanSelector(transactionId);
+ Future<LogRecordWithDLSN> searchResult = asyncReadRecordFromEntries(
+ logName,
+ reader,
+ segment,
+ executorService,
+ new SingleEntryScanContext(entryId),
+ selector);
+ searchResults.add(searchResult);
+ }
+ FutureEventListener<List<LogRecordWithDLSN>> processSearchResultsListener =
+ new FutureEventListener<List<LogRecordWithDLSN>>() {
+ @Override
+ public void onSuccess(List<LogRecordWithDLSN> resultList) {
+ processSearchResults(
+ logName,
+ segment,
+ transactionId,
+ executorService,
+ reader,
+ resultList,
+ nWays,
+ prevFoundRecord,
+ promise);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ promise.setException(cause);
+ }
+ };
+ Future.collect(searchResults).addEventListener(
+ FutureEventListenerRunnable.of(processSearchResultsListener, executorService));
+ }
+
+ /**
+ * Process the search results
+ */
+ static void processSearchResults(
+ final String logName,
+ final LogSegmentMetadata segment,
+ final long transactionId,
+ final ExecutorService executorService,
+ final LogSegmentRandomAccessEntryReader reader,
+ final List<LogRecordWithDLSN> searchResults,
+ final int nWays,
+ final Optional<LogRecordWithDLSN> prevFoundRecord,
+ final Promise<Optional<LogRecordWithDLSN>> promise) {
+ int found = -1;
+ for (int i = 0; i < searchResults.size(); i++) {
+ LogRecordWithDLSN record = searchResults.get(i);
+ if (record.getTransactionId() >= transactionId) {
+ found = i;
+ break;
+ }
+ }
+ if (found == -1) { // all log records' transaction id is less than provided transaction id
+ promise.setValue(prevFoundRecord);
+ return;
+ }
+ // we found a log record
+ LogRecordWithDLSN foundRecord = searchResults.get(found);
+
+ // we found it
+ // - it is not the first record
+ // - it is the first record in first search entry
+ // - its entry is adjacent to previous search entry
+ if (foundRecord.getDlsn().getSlotId() != 0L
+ || found == 0
+ || foundRecord.getDlsn().getEntryId() == (searchResults.get(found - 1).getDlsn().getEntryId() + 1)) {
+ promise.setValue(Optional.of(foundRecord));
+ return;
+ }
+
+ // otherwise, we need to search
+ List<Long> nextSearchBatch = getEntriesToSearch(
+ transactionId,
+ searchResults.get(found - 1),
+ searchResults.get(found),
+ nWays);
+ if (nextSearchBatch.isEmpty()) {
+ promise.setValue(prevFoundRecord);
+ return;
+ }
+ getLogRecordNotLessThanTxIdFromEntries(
+ logName,
+ segment,
+ transactionId,
+ executorService,
+ reader,
+ nextSearchBatch,
+ nWays,
+ Optional.of(foundRecord),
+ promise);
+ }
+
+ /**
+ * Get the entries to search provided <code>transactionId</code> between
+ * <code>firstRecord</code> and <code>lastRecord</code>. <code>firstRecord</code>
+ * and <code>lastRecord</code> are already searched, which the transaction id
+ * of <code>firstRecord</code> is less than <code>transactionId</code> and the
+ * transaction id of <code>lastRecord</code> is not less than <code>transactionId</code>.
+ *
+ * @param transactionId
+ * transaction id to search
+ * @param firstRecord
+ * log record that already searched whose transaction id is leass than <code>transactionId</code>.
+ * @param lastRecord
+ * log record that already searched whose transaction id is not less than <code>transactionId</code>.
+ * @param nWays
+ * N-ways to search
+ * @return the list of entries to search
+ */
+ static List<Long> getEntriesToSearch(
+ long transactionId,
+ LogRecordWithDLSN firstRecord,
+ LogRecordWithDLSN lastRecord,
+ int nWays) {
+ long txnDiff = lastRecord.getTransactionId() - firstRecord.getTransactionId();
+ if (txnDiff > 0) {
+ if (lastRecord.getTransactionId() == transactionId) {
+ List<Long> entries = getEntriesToSearch(
+ firstRecord.getDlsn().getEntryId() + 1,
+ lastRecord.getDlsn().getEntryId() - 2,
+ Math.max(MIN_SEARCH_BATCH_SIZE, nWays - 1));
+ entries.add(lastRecord.getDlsn().getEntryId() - 1);
+ return entries;
+ } else {
+ // TODO: improve it by estimating transaction ids.
+ return getEntriesToSearch(
+ firstRecord.getDlsn().getEntryId() + 1,
+ lastRecord.getDlsn().getEntryId() - 1,
+ nWays);
+ }
+ } else {
+ // unexpected condition
+ return Lists.newArrayList();
+ }
+ }
+
+ static List<Long> getEntriesToSearch(
+ long startEntryId,
+ long endEntryId,
+ int nWays) {
+ if (startEntryId > endEntryId) {
+ return Lists.newArrayList();
+ }
+ long numEntries = endEntryId - startEntryId + 1;
+ long step = Math.max(1L, numEntries / nWays);
+ List<Long> entryList = Lists.newArrayListWithExpectedSize(nWays);
+ for (long i = startEntryId, j = nWays - 1; i <= endEntryId && j > 0; i += step, j--) {
+ entryList.add(i);
+ }
+ if (entryList.get(entryList.size() - 1) < endEntryId) {
+ entryList.add(endEntryId);
+ }
+ return entryList;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
new file mode 100644
index 0000000..d25d056
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
@@ -0,0 +1,62 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.util.PermitLimiter;
+
+public class WriteLimiter {
+
+ String streamName;
+ final PermitLimiter streamLimiter;
+ final PermitLimiter globalLimiter;
+
+ public WriteLimiter(String streamName, PermitLimiter streamLimiter, PermitLimiter globalLimiter) {
+ this.streamName = streamName;
+ this.streamLimiter = streamLimiter;
+ this.globalLimiter = globalLimiter;
+ }
+
+ public void acquire() throws OverCapacityException {
+ if (!streamLimiter.acquire()) {
+ throw new OverCapacityException(String.format("Stream write capacity exceeded for stream %s", streamName));
+ }
+ try {
+ if (!globalLimiter.acquire()) {
+ throw new OverCapacityException("Global write capacity exceeded");
+ }
+ } catch (OverCapacityException ex) {
+ streamLimiter.release(1);
+ throw ex;
+ }
+ }
+
+ public void release() {
+ release(1);
+ }
+
+ public void release(int permits) {
+ streamLimiter.release(permits);
+ globalLimiter.release(permits);
+ }
+
+ public void close() {
+ streamLimiter.close();
+ globalLimiter.close();
+ }
+}
[17/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstRecordSelector.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstRecordSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstRecordSelector.java
deleted file mode 100644
index a51210f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstRecordSelector.java
+++ /dev/null
@@ -1,46 +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.selector;
-
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Save the first record processed
- */
-public class FirstRecordSelector implements LogRecordSelector {
-
- final boolean includeControl;
- LogRecordWithDLSN firstRecord;
-
- public FirstRecordSelector(boolean includeControl) {
- this.includeControl = includeControl;
- }
-
- @Override
- public void process(LogRecordWithDLSN record) {
- if (null == this.firstRecord
- && (includeControl || !record.isControl())) {
- this.firstRecord = record;
- }
- }
-
- @Override
- public LogRecordWithDLSN result() {
- return this.firstRecord;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstTxIdNotLessThanSelector.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstTxIdNotLessThanSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstTxIdNotLessThanSelector.java
deleted file mode 100644
index 03c2cbb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstTxIdNotLessThanSelector.java
+++ /dev/null
@@ -1,51 +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.selector;
-
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Save the first record with transaction id not less than the provided transaction id.
- * If all records' transaction id is less than provided transaction id, save the last record.
- */
-public class FirstTxIdNotLessThanSelector implements LogRecordSelector {
-
- LogRecordWithDLSN result;
- final long txId;
- boolean found = false;
-
- public FirstTxIdNotLessThanSelector(long txId) {
- this.txId = txId;
- }
-
- @Override
- public void process(LogRecordWithDLSN record) {
- if (found) {
- return;
- }
- this.result = record;
- if (record.getTransactionId() >= txId) {
- found = true;
- }
- }
-
- @Override
- public LogRecordWithDLSN result() {
- return this.result;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LastRecordSelector.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LastRecordSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LastRecordSelector.java
deleted file mode 100644
index 191342c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LastRecordSelector.java
+++ /dev/null
@@ -1,38 +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.selector;
-
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Save the last record processed.
- */
-public class LastRecordSelector implements LogRecordSelector {
-
- LogRecordWithDLSN lastRecord;
-
- @Override
- public void process(LogRecordWithDLSN record) {
- lastRecord = record;
- }
-
- @Override
- public LogRecordWithDLSN result() {
- return lastRecord;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LogRecordSelector.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LogRecordSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LogRecordSelector.java
deleted file mode 100644
index 45d1c49..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LogRecordSelector.java
+++ /dev/null
@@ -1,40 +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.selector;
-
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Visitor interface to process a set of records, and return some result.
- */
-public interface LogRecordSelector {
- /**
- * Process a given <code>record</code>.
- *
- * @param record
- * log record to process
- */
- void process(LogRecordWithDLSN record);
-
- /**
- * Returned the selected log record after processing a set of records.
- *
- * @return the selected log record.
- */
- LogRecordWithDLSN result();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BKExceptionStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BKExceptionStatsLogger.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BKExceptionStatsLogger.java
deleted file mode 100644
index be71aef..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BKExceptionStatsLogger.java
+++ /dev/null
@@ -1,109 +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.stats;
-
-import org.apache.bookkeeper.client.BKException.Code;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * A Util to logger stats on bk exceptions.
- */
-public class BKExceptionStatsLogger {
-
- public static String getMessage(int code) {
- switch (code) {
- case Code.OK:
- return "OK";
- case Code.ReadException:
- return "ReadException";
- case Code.QuorumException:
- return "QuorumException";
- case Code.NoBookieAvailableException:
- return "NoBookieAvailableException";
- case Code.DigestNotInitializedException:
- return "DigestNotInitializedException";
- case Code.DigestMatchException:
- return "DigestMatchException";
- case Code.NotEnoughBookiesException:
- return "NotEnoughBookiesException";
- case Code.NoSuchLedgerExistsException:
- return "NoSuchLedgerExistsException";
- case Code.BookieHandleNotAvailableException:
- return "BookieHandleNotAvailableException";
- case Code.ZKException:
- return "ZKException";
- case Code.LedgerRecoveryException:
- return "LedgerRecoveryException";
- case Code.LedgerClosedException:
- return "LedgerClosedException";
- case Code.WriteException:
- return "WriteException";
- case Code.NoSuchEntryException:
- return "NoSuchEntryException";
- case Code.IncorrectParameterException:
- return "IncorrectParameterException";
- case Code.InterruptedException:
- return "InterruptedException";
- case Code.ProtocolVersionException:
- return "ProtocolVersionException";
- case Code.MetadataVersionException:
- return "MetadataVersionException";
- case Code.LedgerFencedException:
- return "LedgerFencedException";
- case Code.UnauthorizedAccessException:
- return "UnauthorizedAccessException";
- case Code.UnclosedFragmentException:
- return "UnclosedFragmentException";
- case Code.WriteOnReadOnlyBookieException:
- return "WriteOnReadOnlyBookieException";
- case Code.IllegalOpException:
- return "IllegalOpException";
- default:
- return "UnexpectedException";
- }
- }
-
- private final StatsLogger parentLogger;
- private final Map<Integer, Counter> exceptionCounters;
-
- public BKExceptionStatsLogger(StatsLogger parentLogger) {
- this.parentLogger = parentLogger;
- this.exceptionCounters = new HashMap<Integer, Counter>();
- }
-
- public Counter getExceptionCounter(int rc) {
- Counter counter = exceptionCounters.get(rc);
- if (null != counter) {
- return counter;
- }
- // TODO: it would be better to have BKException.Code.get(rc)
- synchronized (exceptionCounters) {
- counter = exceptionCounters.get(rc);
- if (null != counter) {
- return counter;
- }
- counter = parentLogger.getCounter(getMessage(rc));
- exceptionCounters.put(rc, counter);
- }
- return counter;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BroadCastStatsLogger.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BroadCastStatsLogger.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BroadCastStatsLogger.java
deleted file mode 100644
index 10a7011..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BroadCastStatsLogger.java
+++ /dev/null
@@ -1,194 +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.stats;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.bookkeeper.stats.CachingStatsLogger;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.OpStatsData;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Stats Loggers that broadcast stats to multiple {@link StatsLogger}.
- */
-public class BroadCastStatsLogger {
-
- /**
- * Create a broadcast stats logger of two stats loggers `<code>first</code>` and
- * `<code>second</code>`. The returned stats logger doesn't allow registering any
- * {@link Gauge}.
- *
- * @param first
- * first stats logger
- * @param second
- * second stats logger
- * @return broadcast stats logger
- */
- public static StatsLogger two(StatsLogger first, StatsLogger second) {
- return new CachingStatsLogger(new Two(first, second));
- }
-
- static class Two implements StatsLogger {
- protected final StatsLogger first;
- protected final StatsLogger second;
-
- private Two(StatsLogger first, StatsLogger second) {
- super();
- Preconditions.checkNotNull(first);
- Preconditions.checkNotNull(second);
- this.first = first;
- this.second = second;
- }
-
- @Override
- public OpStatsLogger getOpStatsLogger(final String statName) {
- final OpStatsLogger firstLogger = first.getOpStatsLogger(statName);
- final OpStatsLogger secondLogger = second.getOpStatsLogger(statName);
- return new OpStatsLogger() {
- @Override
- public void registerFailedEvent(long l) {
- firstLogger.registerFailedEvent(l);
- secondLogger.registerFailedEvent(l);
- }
-
- @Override
- public void registerSuccessfulEvent(long l) {
- firstLogger.registerSuccessfulEvent(l);
- secondLogger.registerSuccessfulEvent(l);
- }
-
- @Override
- public OpStatsData toOpStatsData() {
- // Eventually consistent.
- return firstLogger.toOpStatsData();
- }
-
- @Override
- public void clear() {
- firstLogger.clear();
- secondLogger.clear();
- }
- };
- }
-
- @Override
- public Counter getCounter(final String statName) {
- final Counter firstCounter = first.getCounter(statName);
- final Counter secondCounter = second.getCounter(statName);
- return new Counter() {
- @Override
- public void clear() {
- firstCounter.clear();
- secondCounter.clear();
- }
-
- @Override
- public void inc() {
- firstCounter.inc();
- secondCounter.inc();
- }
-
- @Override
- public void dec() {
- firstCounter.dec();
- secondCounter.dec();
- }
-
- @Override
- public void add(long l) {
- firstCounter.add(l);
- secondCounter.add(l);
- }
-
- @Override
- public Long get() {
- // Eventually consistent.
- return firstCounter.get();
- }
- };
- }
-
- @Override
- public <T extends Number> void registerGauge(String statName, Gauge<T> gauge) {
- // Different underlying stats loggers have different semantics wrt. gauge registration.
- throw new RuntimeException("Cannot register a gauge on BroadCastStatsLogger.Two");
- }
-
- @Override
- public <T extends Number> void unregisterGauge(String statName, Gauge<T> gauge) {
- // no-op
- }
-
- @Override
- public StatsLogger scope(final String scope) {
- return new Two(first.scope(scope), second.scope(scope));
- }
-
- @Override
- public void removeScope(String scope, StatsLogger statsLogger) {
- if (!(statsLogger instanceof Two)) {
- return;
- }
-
- Two another = (Two) statsLogger;
-
- first.removeScope(scope, another.first);
- second.removeScope(scope, another.second);
- }
- }
-
- /**
- * Create a broadcast stats logger of two stats loggers <code>master</code> and <code>slave</code>.
- * It is similar as {@link #two(StatsLogger, StatsLogger)}, but it allows registering {@link Gauge}s.
- * The {@link Gauge} will be registered under master.
- *
- * @param master
- * master stats logger to receive {@link Counter}, {@link OpStatsLogger} and {@link Gauge}.
- * @param slave
- * slave stats logger to receive only {@link Counter} and {@link OpStatsLogger}.
- * @return broadcast stats logger
- */
- public static StatsLogger masterslave(StatsLogger master, StatsLogger slave) {
- return new CachingStatsLogger(new MasterSlave(master, slave));
- }
-
- static class MasterSlave extends Two {
-
- private MasterSlave(StatsLogger master, StatsLogger slave) {
- super(master, slave);
- }
-
- @Override
- public <T extends Number> void registerGauge(String statName, Gauge<T> gauge) {
- first.registerGauge(statName, gauge);
- }
-
- @Override
- public <T extends Number> void unregisterGauge(String statName, Gauge<T> gauge) {
- first.unregisterGauge(statName, gauge);
- }
-
- @Override
- public StatsLogger scope(String scope) {
- return new MasterSlave(first.scope(scope), second.scope(scope));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/OpStatsListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/OpStatsListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/OpStatsListener.java
deleted file mode 100644
index 0432706..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/OpStatsListener.java
+++ /dev/null
@@ -1,51 +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.stats;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.util.FutureEventListener;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import java.util.concurrent.TimeUnit;
-
-public class OpStatsListener<T> implements FutureEventListener<T> {
- OpStatsLogger opStatsLogger;
- Stopwatch stopwatch;
-
- public OpStatsListener(OpStatsLogger opStatsLogger, Stopwatch stopwatch) {
- this.opStatsLogger = opStatsLogger;
- if (null == stopwatch) {
- this.stopwatch = Stopwatch.createStarted();
- } else {
- this.stopwatch = stopwatch;
- }
- }
-
- public OpStatsListener(OpStatsLogger opStatsLogger) {
- this(opStatsLogger, null);
- }
-
- @Override
- public void onSuccess(T value) {
- opStatsLogger.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
- }
-
- @Override
- public void onFailure(Throwable cause) {
- opStatsLogger.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionStateStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionStateStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionStateStore.java
deleted file mode 100644
index 9e4c4f2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionStateStore.java
+++ /dev/null
@@ -1,42 +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.subscription;
-
-import java.io.Closeable;
-
-import scala.runtime.BoxedUnit;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.util.Future;
-
-public interface SubscriptionStateStore extends Closeable {
- /**
- * Get the last committed position stored for this subscription
- *
- * @return future represents the last commit position
- */
- public Future<DLSN> getLastCommitPosition();
-
- /**
- * Advances the position associated with the subscriber
- *
- * @param newPosition - new commit position
- * @return future represents the advance result
- */
- public Future<BoxedUnit> advanceCommitPosition(DLSN newPosition);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionsStore.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionsStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionsStore.java
deleted file mode 100644
index 27d5c1d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionsStore.java
+++ /dev/null
@@ -1,69 +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.subscription;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.util.Future;
-import scala.runtime.BoxedUnit;
-
-import java.io.Closeable;
-import java.util.Map;
-
-/**
- * Store to manage subscriptions
- */
-public interface SubscriptionsStore extends Closeable {
-
- /**
- * Get the last committed position stored for <i>subscriberId</i>.
- *
- * @param subscriberId
- * subscriber id
- * @return future representing last committed position.
- */
- public Future<DLSN> getLastCommitPosition(String subscriberId);
-
- /**
- * Get the last committed positions for all subscribers.
- *
- * @return future representing last committed positions for all subscribers.
- */
- public Future<Map<String, DLSN>> getLastCommitPositions();
-
- /**
- * Advance the last committed position for <i>subscriberId</i>.
- *
- * @param subscriberId
- * subscriber id.
- * @param newPosition
- * new committed position.
- * @return future representing advancing result.
- */
- public Future<BoxedUnit> advanceCommitPosition(String subscriberId, DLSN newPosition);
-
- /**
- * Delete the subscriber <i>subscriberId</i> permanently. Once the subscriber is deleted, all the
- * data stored under this subscriber will be lost.
- * @param subscriberId subscriber id
- * @return future represent success or failure.
- * return true only if there's such subscriber and we removed it successfully.
- * return false if there's no such subscriber, or we failed to remove.
- */
- public Future<Boolean> deleteSubscriber(String subscriberId);
-
-}
[37/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
deleted file mode 100644
index aee4103..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
+++ /dev/null
@@ -1,751 +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.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.google.common.base.Ticker;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.IdleReaderException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.ReadCancelledException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Throw;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Function1;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-/**
- * BookKeeper based {@link AsyncLogReader} implementation.
- *
- * <h3>Metrics</h3>
- * All the metrics are exposed under `async_reader`.
- * <ul>
- * <li> `async_reader`/future_set: opstats. time spent on satisfying futures of read requests.
- * if it is high, it means that the caller takes time on processing the result of read requests.
- * The side effect is blocking consequent reads.
- * <li> `async_reader`/schedule: opstats. time spent on scheduling next reads.
- * <li> `async_reader`/background_read: opstats. time spent on background reads.
- * <li> `async_reader`/read_next_exec: opstats. time spent on executing {@link #readNext()}.
- * <li> `async_reader`/time_between_read_next: opstats. time spent on between two consequent {@link #readNext()}.
- * if it is high, it means that the caller is slowing down on calling {@link #readNext()}.
- * <li> `async_reader`/delay_until_promise_satisfied: opstats. total latency for the read requests.
- * <li> `async_reader`/idle_reader_error: counter. the number idle reader errors.
- * </ul>
- */
-class BKAsyncLogReader implements AsyncLogReader, Runnable, AsyncNotification {
- static final Logger LOG = LoggerFactory.getLogger(BKAsyncLogReader.class);
-
- private static final Function1<List<LogRecordWithDLSN>, LogRecordWithDLSN> READ_NEXT_MAP_FUNCTION =
- new AbstractFunction1<List<LogRecordWithDLSN>, LogRecordWithDLSN>() {
- @Override
- public LogRecordWithDLSN apply(List<LogRecordWithDLSN> records) {
- return records.get(0);
- }
- };
-
- private final String streamName;
- protected final BKDistributedLogManager bkDistributedLogManager;
- protected final BKLogReadHandler readHandler;
- private final AtomicReference<Throwable> lastException = new AtomicReference<Throwable>();
- private final OrderedScheduler scheduler;
- private final ConcurrentLinkedQueue<PendingReadRequest> pendingRequests = new ConcurrentLinkedQueue<PendingReadRequest>();
- private final Object scheduleLock = new Object();
- private final AtomicLong scheduleCount = new AtomicLong(0);
- final private Stopwatch scheduleDelayStopwatch;
- final private Stopwatch readNextDelayStopwatch;
- private DLSN startDLSN;
- private ReadAheadEntryReader readAheadReader = null;
- private int lastPosition = 0;
- private final boolean positionGapDetectionEnabled;
- private final int idleErrorThresholdMillis;
- final ScheduledFuture<?> idleReaderTimeoutTask;
- private ScheduledFuture<?> backgroundScheduleTask = null;
- // last process time
- private final Stopwatch lastProcessTime;
-
- protected Promise<Void> closeFuture = null;
-
- private boolean lockStream = false;
-
- private final boolean returnEndOfStreamRecord;
-
- private final Runnable BACKGROUND_READ_SCHEDULER = new Runnable() {
- @Override
- public void run() {
- synchronized (scheduleLock) {
- backgroundScheduleTask = null;
- }
- scheduleBackgroundRead();
- }
- };
-
- // State
- private Entry.Reader currentEntry = null;
- private LogRecordWithDLSN nextRecord = null;
-
- // Failure Injector
- private boolean disableProcessingReadRequests = false;
-
- // Stats
- private final OpStatsLogger readNextExecTime;
- private final OpStatsLogger delayUntilPromiseSatisfied;
- private final OpStatsLogger timeBetweenReadNexts;
- private final OpStatsLogger futureSetLatency;
- private final OpStatsLogger scheduleLatency;
- private final OpStatsLogger backgroundReaderRunTime;
- private final Counter idleReaderCheckCount;
- private final Counter idleReaderCheckIdleReadRequestCount;
- private final Counter idleReaderCheckIdleReadAheadCount;
- private final Counter idleReaderError;
-
- private class PendingReadRequest {
- private final Stopwatch enqueueTime;
- private final int numEntries;
- private final List<LogRecordWithDLSN> records;
- private final Promise<List<LogRecordWithDLSN>> promise;
- private final long deadlineTime;
- private final TimeUnit deadlineTimeUnit;
-
- PendingReadRequest(int numEntries,
- long deadlineTime,
- TimeUnit deadlineTimeUnit) {
- this.numEntries = numEntries;
- this.enqueueTime = Stopwatch.createStarted();
- // optimize the space usage for single read.
- if (numEntries == 1) {
- this.records = new ArrayList<LogRecordWithDLSN>(1);
- } else {
- this.records = new ArrayList<LogRecordWithDLSN>();
- }
- this.promise = new Promise<List<LogRecordWithDLSN>>();
- this.deadlineTime = deadlineTime;
- this.deadlineTimeUnit = deadlineTimeUnit;
- }
-
- Promise<List<LogRecordWithDLSN>> getPromise() {
- return promise;
- }
-
- long elapsedSinceEnqueue(TimeUnit timeUnit) {
- return enqueueTime.elapsed(timeUnit);
- }
-
- void setException(Throwable throwable) {
- Stopwatch stopwatch = Stopwatch.createStarted();
- if (promise.updateIfEmpty(new Throw<List<LogRecordWithDLSN>>(throwable))) {
- futureSetLatency.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- delayUntilPromiseSatisfied.registerFailedEvent(enqueueTime.elapsed(TimeUnit.MICROSECONDS));
- }
- }
-
- boolean hasReadRecords() {
- return records.size() > 0;
- }
-
- boolean hasReadEnoughRecords() {
- return records.size() >= numEntries;
- }
-
- long getRemainingWaitTime() {
- if (deadlineTime <= 0L) {
- return 0L;
- }
- return deadlineTime - elapsedSinceEnqueue(deadlineTimeUnit);
- }
-
- void addRecord(LogRecordWithDLSN record) {
- records.add(record);
- }
-
- void complete() {
- if (LOG.isTraceEnabled()) {
- LOG.trace("{} : Satisfied promise with {} records", readHandler.getFullyQualifiedName(), records.size());
- }
- delayUntilPromiseSatisfied.registerSuccessfulEvent(enqueueTime.stop().elapsed(TimeUnit.MICROSECONDS));
- Stopwatch stopwatch = Stopwatch.createStarted();
- promise.setValue(records);
- futureSetLatency.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- }
- }
-
- BKAsyncLogReader(BKDistributedLogManager bkdlm,
- OrderedScheduler scheduler,
- DLSN startDLSN,
- Optional<String> subscriberId,
- boolean returnEndOfStreamRecord,
- StatsLogger statsLogger) {
- this.streamName = bkdlm.getStreamName();
- this.bkDistributedLogManager = bkdlm;
- this.scheduler = scheduler;
- this.readHandler = bkDistributedLogManager.createReadHandler(subscriberId,
- this, true);
- LOG.debug("Starting async reader at {}", startDLSN);
- this.startDLSN = startDLSN;
- this.scheduleDelayStopwatch = Stopwatch.createUnstarted();
- this.readNextDelayStopwatch = Stopwatch.createStarted();
- this.positionGapDetectionEnabled = bkdlm.getConf().getPositionGapDetectionEnabled();
- this.idleErrorThresholdMillis = bkdlm.getConf().getReaderIdleErrorThresholdMillis();
- this.returnEndOfStreamRecord = returnEndOfStreamRecord;
-
- // Stats
- StatsLogger asyncReaderStatsLogger = statsLogger.scope("async_reader");
- futureSetLatency = asyncReaderStatsLogger.getOpStatsLogger("future_set");
- scheduleLatency = asyncReaderStatsLogger.getOpStatsLogger("schedule");
- backgroundReaderRunTime = asyncReaderStatsLogger.getOpStatsLogger("background_read");
- readNextExecTime = asyncReaderStatsLogger.getOpStatsLogger("read_next_exec");
- timeBetweenReadNexts = asyncReaderStatsLogger.getOpStatsLogger("time_between_read_next");
- delayUntilPromiseSatisfied = asyncReaderStatsLogger.getOpStatsLogger("delay_until_promise_satisfied");
- idleReaderError = asyncReaderStatsLogger.getCounter("idle_reader_error");
- idleReaderCheckCount = asyncReaderStatsLogger.getCounter("idle_reader_check_total");
- idleReaderCheckIdleReadRequestCount = asyncReaderStatsLogger.getCounter("idle_reader_check_idle_read_requests");
- idleReaderCheckIdleReadAheadCount = asyncReaderStatsLogger.getCounter("idle_reader_check_idle_readahead");
-
- // Lock the stream if requested. The lock will be released when the reader is closed.
- this.lockStream = false;
- this.idleReaderTimeoutTask = scheduleIdleReaderTaskIfNecessary();
- this.lastProcessTime = Stopwatch.createStarted();
- }
-
- private ScheduledFuture<?> scheduleIdleReaderTaskIfNecessary() {
- if (idleErrorThresholdMillis < Integer.MAX_VALUE) {
- // Dont run the task more than once every seconds (for sanity)
- long period = Math.max(idleErrorThresholdMillis / 10, 1000);
- // Except when idle reader threshold is less than a second (tests?)
- period = Math.min(period, idleErrorThresholdMillis / 5);
-
- return scheduler.scheduleAtFixedRate(streamName, new Runnable() {
- @Override
- public void run() {
- PendingReadRequest nextRequest = pendingRequests.peek();
-
- idleReaderCheckCount.inc();
- if (null == nextRequest) {
- return;
- }
-
- idleReaderCheckIdleReadRequestCount.inc();
- if (nextRequest.elapsedSinceEnqueue(TimeUnit.MILLISECONDS) < idleErrorThresholdMillis) {
- return;
- }
-
- ReadAheadEntryReader readAheadReader = getReadAheadReader();
-
- // read request has been idle
- // - cache has records but read request are idle,
- // that means notification was missed between readahead and reader.
- // - cache is empty and readahead is idle (no records added for a long time)
- idleReaderCheckIdleReadAheadCount.inc();
- try {
- if (null == readAheadReader || (!hasMoreRecords() &&
- readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS))) {
- markReaderAsIdle();
- return;
- } else if (lastProcessTime.elapsed(TimeUnit.MILLISECONDS) > idleErrorThresholdMillis) {
- markReaderAsIdle();;
- }
- } catch (IOException e) {
- setLastException(e);
- return;
- }
- }
- }, period, period, TimeUnit.MILLISECONDS);
- }
- return null;
- }
-
- synchronized ReadAheadEntryReader getReadAheadReader() {
- return readAheadReader;
- }
-
- void cancelIdleReaderTask() {
- // Do this after we have checked that the reader was not previously closed
- try {
- if (null != idleReaderTimeoutTask) {
- idleReaderTimeoutTask.cancel(true);
- }
- } catch (Exception exc) {
- LOG.info("{}: Failed to cancel the background idle reader timeout task", readHandler.getFullyQualifiedName());
- }
- }
-
- private void markReaderAsIdle() {
- idleReaderError.inc();
- IdleReaderException ire = new IdleReaderException("Reader on stream "
- + readHandler.getFullyQualifiedName()
- + " is idle for " + idleErrorThresholdMillis +" ms");
- setLastException(ire);
- // cancel all pending reads directly rather than notifying on error
- // because idle reader could happen on idle read requests that usually means something wrong
- // in scheduling reads
- cancelAllPendingReads(ire);
- }
-
- protected synchronized void setStartDLSN(DLSN fromDLSN) throws UnexpectedException {
- if (null != readAheadReader) {
- throw new UnexpectedException("Could't reset from dlsn after reader already starts reading.");
- }
- startDLSN = fromDLSN;
- }
-
- @VisibleForTesting
- public synchronized DLSN getStartDLSN() {
- return startDLSN;
- }
-
- public Future<Void> lockStream() {
- this.lockStream = true;
- return readHandler.lockStream();
- }
-
- private boolean checkClosedOrInError(String operation) {
- if (null == lastException.get()) {
- try {
- if (null != readHandler && null != getReadAheadReader()) {
- getReadAheadReader().checkLastException();
- }
-
- bkDistributedLogManager.checkClosedOrInError(operation);
- } catch (IOException exc) {
- setLastException(exc);
- }
- }
-
- if (lockStream) {
- try {
- readHandler.checkReadLock();
- } catch (IOException ex) {
- setLastException(ex);
- }
- }
-
- if (null != lastException.get()) {
- LOG.trace("Cancelling pending reads");
- cancelAllPendingReads(lastException.get());
- return true;
- }
-
- return false;
- }
-
- private void setLastException(IOException exc) {
- lastException.compareAndSet(null, exc);
- }
-
- @Override
- public String getStreamName() {
- return streamName;
- }
-
- /**
- * @return A promise that when satisfied will contain the Log Record with its DLSN.
- */
- @Override
- public synchronized Future<LogRecordWithDLSN> readNext() {
- return readInternal(1, 0, TimeUnit.MILLISECONDS).map(READ_NEXT_MAP_FUNCTION);
- }
-
- public synchronized Future<List<LogRecordWithDLSN>> readBulk(int numEntries) {
- return readInternal(numEntries, 0, TimeUnit.MILLISECONDS);
- }
-
- @Override
- public synchronized Future<List<LogRecordWithDLSN>> readBulk(int numEntries,
- long waitTime,
- TimeUnit timeUnit) {
- return readInternal(numEntries, waitTime, timeUnit);
- }
-
- /**
- * Read up to <i>numEntries</i> entries. The future will be satisfied when any number of entries are
- * ready (1 to <i>numEntries</i>).
- *
- * @param numEntries
- * num entries to read
- * @return A promise that satisfied with a non-empty list of log records with their DLSN.
- */
- private synchronized Future<List<LogRecordWithDLSN>> readInternal(int numEntries,
- long deadlineTime,
- TimeUnit deadlineTimeUnit) {
- timeBetweenReadNexts.registerSuccessfulEvent(readNextDelayStopwatch.elapsed(TimeUnit.MICROSECONDS));
- readNextDelayStopwatch.reset().start();
- final PendingReadRequest readRequest = new PendingReadRequest(numEntries, deadlineTime, deadlineTimeUnit);
-
- if (null == readAheadReader) {
- final ReadAheadEntryReader readAheadEntryReader = this.readAheadReader = new ReadAheadEntryReader(
- getStreamName(),
- getStartDLSN(),
- bkDistributedLogManager.getConf(),
- readHandler,
- bkDistributedLogManager.getReaderEntryStore(),
- bkDistributedLogManager.getScheduler(),
- Ticker.systemTicker(),
- bkDistributedLogManager.alertStatsLogger);
- readHandler.checkLogStreamExists().addEventListener(new FutureEventListener<Void>() {
- @Override
- public void onSuccess(Void value) {
- try {
- readHandler.registerListener(readAheadEntryReader);
- readHandler.asyncStartFetchLogSegments()
- .map(new AbstractFunction1<Versioned<List<LogSegmentMetadata>>, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Versioned<List<LogSegmentMetadata>> logSegments) {
- readAheadEntryReader.addStateChangeNotification(BKAsyncLogReader.this);
- readAheadEntryReader.start(logSegments.getValue());
- return BoxedUnit.UNIT;
- }
- });
- } catch (Exception exc) {
- notifyOnError(exc);
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- notifyOnError(cause);
- }
- });
- }
-
- if (checkClosedOrInError("readNext")) {
- readRequest.setException(lastException.get());
- } else {
- boolean queueEmpty = pendingRequests.isEmpty();
- pendingRequests.add(readRequest);
-
- if (queueEmpty) {
- scheduleBackgroundRead();
- }
- }
-
- readNextExecTime.registerSuccessfulEvent(readNextDelayStopwatch.elapsed(TimeUnit.MICROSECONDS));
- readNextDelayStopwatch.reset().start();
-
- return readRequest.getPromise();
- }
-
- public synchronized void scheduleBackgroundRead() {
- // if the reader is already closed, we don't need to schedule background read again.
- if (null != closeFuture) {
- return;
- }
-
- long prevCount = scheduleCount.getAndIncrement();
- if (0 == prevCount) {
- scheduleDelayStopwatch.reset().start();
- scheduler.submit(streamName, this);
- }
- }
-
- @Override
- public Future<Void> asyncClose() {
- // Cancel the idle reader timeout task, interrupting if necessary
- ReadCancelledException exception;
- Promise<Void> closePromise;
- synchronized (this) {
- if (null != closeFuture) {
- return closeFuture;
- }
- closePromise = closeFuture = new Promise<Void>();
- exception = new ReadCancelledException(readHandler.getFullyQualifiedName(), "Reader was closed");
- setLastException(exception);
- }
-
- // Do this after we have checked that the reader was not previously closed
- cancelIdleReaderTask();
-
- synchronized (scheduleLock) {
- if (null != backgroundScheduleTask) {
- backgroundScheduleTask.cancel(true);
- }
- }
-
- cancelAllPendingReads(exception);
-
- ReadAheadEntryReader readAheadReader = getReadAheadReader();
- if (null != readAheadReader) {
- readHandler.unregisterListener(readAheadReader);
- readAheadReader.removeStateChangeNotification(this);
- }
- Utils.closeSequence(bkDistributedLogManager.getScheduler(), true,
- readAheadReader,
- readHandler
- ).proxyTo(closePromise);
- return closePromise;
- }
-
- private void cancelAllPendingReads(Throwable throwExc) {
- for (PendingReadRequest promise : pendingRequests) {
- promise.setException(throwExc);
- }
- pendingRequests.clear();
- }
-
- synchronized boolean hasMoreRecords() throws IOException {
- if (null == readAheadReader) {
- return false;
- }
- if (readAheadReader.getNumCachedEntries() > 0 || null != nextRecord) {
- return true;
- } else if (null != currentEntry) {
- nextRecord = currentEntry.nextRecord();
- return null != nextRecord;
- }
- return false;
- }
-
- private synchronized LogRecordWithDLSN readNextRecord() throws IOException {
- if (null == readAheadReader) {
- return null;
- }
- if (null == currentEntry) {
- currentEntry = readAheadReader.getNextReadAheadEntry(0L, TimeUnit.MILLISECONDS);
- // no entry after reading from read ahead then return null
- if (null == currentEntry) {
- return null;
- }
- }
-
- LogRecordWithDLSN recordToReturn;
- if (null == nextRecord) {
- nextRecord = currentEntry.nextRecord();
- // no more records in current entry
- if (null == nextRecord) {
- currentEntry = null;
- return readNextRecord();
- }
- }
-
- // found a record to return and prefetch the next one
- recordToReturn = nextRecord;
- nextRecord = currentEntry.nextRecord();
- return recordToReturn;
- }
-
- @Override
- public void run() {
- synchronized(scheduleLock) {
- if (scheduleDelayStopwatch.isRunning()) {
- scheduleLatency.registerSuccessfulEvent(scheduleDelayStopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
- }
-
- Stopwatch runTime = Stopwatch.createStarted();
- int iterations = 0;
- long scheduleCountLocal = scheduleCount.get();
- LOG.debug("{}: Scheduled Background Reader", readHandler.getFullyQualifiedName());
- while(true) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("{}: Executing Iteration: {}", readHandler.getFullyQualifiedName(), iterations++);
- }
-
- PendingReadRequest nextRequest = null;
- synchronized(this) {
- nextRequest = pendingRequests.peek();
-
- // Queue is empty, nothing to read, return
- if (null == nextRequest) {
- LOG.trace("{}: Queue Empty waiting for Input", readHandler.getFullyQualifiedName());
- scheduleCount.set(0);
- backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
- return;
- }
-
- if (disableProcessingReadRequests) {
- LOG.info("Reader of {} is forced to stop processing read requests", readHandler.getFullyQualifiedName());
- return;
- }
- }
- 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
- if (null == lastException.get()) {
- if (nextRequest.getPromise().isInterrupted().isDefined()) {
- setLastException(new DLInterruptedException("Interrupted on reading " + readHandler.getFullyQualifiedName() + " : ",
- nextRequest.getPromise().isInterrupted().get()));
- }
- }
-
- if (checkClosedOrInError("readNext")) {
- if (!(lastException.get().getCause() instanceof LogNotFoundException)) {
- LOG.warn("{}: Exception", readHandler.getFullyQualifiedName(), lastException.get());
- }
- backgroundReaderRunTime.registerFailedEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
- return;
- }
-
- try {
- // Fail 10% of the requests when asked to simulate errors
- if (bkDistributedLogManager.getFailureInjector().shouldInjectErrors()) {
- throw new IOException("Reader Simulated Exception");
- }
- LogRecordWithDLSN record;
- while (!nextRequest.hasReadEnoughRecords()) {
- // read single record
- do {
- record = readNextRecord();
- } while (null != record && (record.isControl() || (record.getDlsn().compareTo(getStartDLSN()) < 0)));
- if (null == record) {
- break;
- } else {
- if (record.isEndOfStream() && !returnEndOfStreamRecord) {
- setLastException(new EndOfStreamException("End of Stream Reached for "
- + readHandler.getFullyQualifiedName()));
- break;
- }
-
- // gap detection
- if (recordPositionsContainsGap(record, lastPosition)) {
- bkDistributedLogManager.raiseAlert("Gap detected between records at record = {}", record);
- if (positionGapDetectionEnabled) {
- throw new DLIllegalStateException("Gap detected between records at record = " + record);
- }
- }
- lastPosition = record.getLastPositionWithinLogSegment();
-
- nextRequest.addRecord(record);
- }
- };
- } catch (IOException exc) {
- setLastException(exc);
- if (!(exc instanceof LogNotFoundException)) {
- LOG.warn("{} : read with skip Exception", readHandler.getFullyQualifiedName(), lastException.get());
- }
- continue;
- }
-
- if (nextRequest.hasReadRecords()) {
- long remainingWaitTime = nextRequest.getRemainingWaitTime();
- if (remainingWaitTime > 0 && !nextRequest.hasReadEnoughRecords()) {
- backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
- scheduleDelayStopwatch.reset().start();
- scheduleCount.set(0);
- // the request could still wait for more records
- backgroundScheduleTask = scheduler.schedule(
- streamName,
- BACKGROUND_READ_SCHEDULER,
- remainingWaitTime,
- nextRequest.deadlineTimeUnit);
- return;
- }
-
- PendingReadRequest request = pendingRequests.poll();
- if (null != request && nextRequest == request) {
- request.complete();
- if (null != backgroundScheduleTask) {
- backgroundScheduleTask.cancel(true);
- backgroundScheduleTask = null;
- }
- } else {
- DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at dlsn = "
- + nextRequest.records.get(0).getDlsn());
- nextRequest.setException(ise);
- if (null != request) {
- request.setException(ise);
- }
- // We should never get here as we should have exited the loop if
- // pendingRequests were empty
- bkDistributedLogManager.raiseAlert("Unexpected condition at dlsn = {}",
- nextRequest.records.get(0).getDlsn());
- setLastException(ise);
- }
- } else {
- if (0 == scheduleCountLocal) {
- LOG.trace("Schedule count dropping to zero", lastException.get());
- backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
- return;
- }
- scheduleCountLocal = scheduleCount.decrementAndGet();
- }
- }
- }
- }
-
- private boolean recordPositionsContainsGap(LogRecordWithDLSN record, long lastPosition) {
- final boolean firstLogRecord = (1 == record.getPositionWithinLogSegment());
- final boolean endOfStreamRecord = record.isEndOfStream();
- final boolean emptyLogSegment = (0 == lastPosition);
- final boolean positionIncreasedByOne = (record.getPositionWithinLogSegment() == (lastPosition + 1));
-
- return !firstLogRecord && !endOfStreamRecord && !emptyLogSegment &&
- !positionIncreasedByOne;
- }
-
- /**
- * Triggered when the background activity encounters an exception
- */
- @Override
- public void notifyOnError(Throwable cause) {
- if (cause instanceof IOException) {
- setLastException((IOException) cause);
- } else {
- setLastException(new IOException(cause));
- }
- scheduleBackgroundRead();
- }
-
- /**
- * Triggered when the background activity completes an operation
- */
- @Override
- public void notifyOnOperationComplete() {
- scheduleBackgroundRead();
- }
-
- @VisibleForTesting
- void simulateErrors() {
- bkDistributedLogManager.getFailureInjector().injectErrors(true);
- }
-
- @VisibleForTesting
- synchronized void disableReadAheadLogSegmentsNotification() {
- readHandler.disableReadAheadLogSegmentsNotification();
- }
-
- @VisibleForTesting
- synchronized void disableProcessingReadRequests() {
- disableProcessingReadRequests = true;
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
deleted file mode 100644
index 9432e8a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
+++ /dev/null
@@ -1,559 +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.Stopwatch;
-import com.google.common.annotations.VisibleForTesting;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.StreamNotReadyException;
-import com.twitter.distributedlog.exceptions.WriteCancelledException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Try;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Function1;
-import scala.Option;
-import scala.runtime.AbstractFunction1;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * BookKeeper based {@link AsyncLogWriter} implementation.
- *
- * <h3>Metrics</h3>
- * All the metrics are exposed under `log_writer`.
- * <ul>
- * <li> `log_writer/write`: opstats. latency characteristics about the time that write operations spent.
- * <li> `log_writer/bulk_write`: opstats. latency characteristics about the time that bulk_write
- * operations spent.
- * are pending in the queue for long time due to log segment rolling.
- * <li> `log_writer/get_writer`: opstats. the time spent on getting the writer. it could spike when there
- * is log segment rolling happened during getting the writer. it is a good stat to look into when the latency
- * is caused by queuing time.
- * <li> `log_writer/pending_request_dispatch`: counter. the number of queued operations that are dispatched
- * after log segment is rolled. it is an metric on measuring how many operations has been queued because of
- * log segment rolling.
- * </ul>
- * See {@link BKLogSegmentWriter} for segment writer stats.
- */
-public class BKAsyncLogWriter extends BKAbstractLogWriter implements AsyncLogWriter {
-
- static final Logger LOG = LoggerFactory.getLogger(BKAsyncLogWriter.class);
-
- static Function1<List<LogSegmentMetadata>, Boolean> TruncationResultConverter =
- new AbstractFunction1<List<LogSegmentMetadata>, Boolean>() {
- @Override
- public Boolean apply(List<LogSegmentMetadata> segments) {
- return true;
- }
- };
-
- // Records pending for roll log segment.
- class PendingLogRecord implements FutureEventListener<DLSN> {
-
- final LogRecord record;
- final Promise<DLSN> promise;
- final boolean flush;
-
- PendingLogRecord(LogRecord record, boolean flush) {
- this.record = record;
- this.promise = new Promise<DLSN>();
- this.flush = flush;
- }
-
- @Override
- public void onSuccess(DLSN value) {
- promise.setValue(value);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- promise.setException(cause);
- encounteredError = true;
- }
- }
-
- /**
- * Last pending record in current log segment. After it is satisified, it would
- * roll log segment.
- *
- * This implementation is based on the assumption that all future satisified in same
- * order future pool.
- */
- class LastPendingLogRecord extends PendingLogRecord {
-
- LastPendingLogRecord(LogRecord record, boolean flush) {
- super(record, flush);
- }
-
- @Override
- public void onSuccess(DLSN value) {
- super.onSuccess(value);
- // roll log segment and issue all pending requests.
- rollLogSegmentAndIssuePendingRequests(record.getTransactionId());
- }
-
- @Override
- public void onFailure(Throwable cause) {
- super.onFailure(cause);
- // error out pending requests.
- errorOutPendingRequestsAndWriter(cause);
- }
- }
-
- private final boolean streamFailFast;
- private final boolean disableRollOnSegmentError;
- private LinkedList<PendingLogRecord> pendingRequests = null;
- private volatile boolean encounteredError = false;
- private Promise<BKLogSegmentWriter> rollingFuture = null;
- private long lastTxId = DistributedLogConstants.INVALID_TXID;
-
- private final StatsLogger statsLogger;
- private final OpStatsLogger writeOpStatsLogger;
- private final OpStatsLogger markEndOfStreamOpStatsLogger;
- private final OpStatsLogger bulkWriteOpStatsLogger;
- private final OpStatsLogger getWriterOpStatsLogger;
- private final Counter pendingRequestDispatch;
-
- private final Feature disableLogSegmentRollingFeature;
-
- BKAsyncLogWriter(DistributedLogConfiguration conf,
- DynamicDistributedLogConfiguration dynConf,
- BKDistributedLogManager bkdlm,
- BKLogWriteHandler writeHandler, /** log writer owns the handler **/
- FeatureProvider featureProvider,
- StatsLogger dlmStatsLogger) {
- super(conf, dynConf, bkdlm);
- this.writeHandler = writeHandler;
- this.streamFailFast = conf.getFailFastOnStreamNotReady();
- this.disableRollOnSegmentError = conf.getDisableRollingOnLogSegmentError();
-
- // features
- disableLogSegmentRollingFeature = featureProvider.getFeature(CoreFeatureKeys.DISABLE_LOGSEGMENT_ROLLING.name().toLowerCase());
- // stats
- this.statsLogger = dlmStatsLogger.scope("log_writer");
- this.writeOpStatsLogger = statsLogger.getOpStatsLogger("write");
- this.markEndOfStreamOpStatsLogger = statsLogger.getOpStatsLogger("mark_end_of_stream");
- this.bulkWriteOpStatsLogger = statsLogger.getOpStatsLogger("bulk_write");
- this.getWriterOpStatsLogger = statsLogger.getOpStatsLogger("get_writer");
- this.pendingRequestDispatch = statsLogger.getCounter("pending_request_dispatch");
- }
-
- @VisibleForTesting
- synchronized void setLastTxId(long txId) {
- lastTxId = Math.max(lastTxId, txId);
- }
-
- @Override
- public synchronized long getLastTxId() {
- return lastTxId;
- }
-
- /**
- * Write a log record as control record. The method will be used by Monitor Service to enforce a new inprogress segment.
- *
- * @param record
- * log record
- * @return future of the write
- */
- public Future<DLSN> writeControlRecord(final LogRecord record) {
- record.setControl();
- return write(record);
- }
-
- private BKLogSegmentWriter getCachedLogSegmentWriter() throws WriteException {
- if (encounteredError) {
- throw new WriteException(bkDistributedLogManager.getStreamName(),
- "writer has been closed due to error.");
- }
- BKLogSegmentWriter segmentWriter = getCachedLogWriter();
- if (null != segmentWriter
- && segmentWriter.isLogSegmentInError()
- && !disableRollOnSegmentError) {
- return null;
- } else {
- return segmentWriter;
- }
- }
-
- private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
- boolean bestEffort,
- boolean rollLog,
- boolean allowMaxTxID) {
- Stopwatch stopwatch = Stopwatch.createStarted();
- return FutureUtils.stats(
- doGetLogSegmentWriter(firstTxid, bestEffort, rollLog, allowMaxTxID),
- getWriterOpStatsLogger,
- stopwatch);
- }
-
- private Future<BKLogSegmentWriter> doGetLogSegmentWriter(final long firstTxid,
- final boolean bestEffort,
- final boolean rollLog,
- final boolean allowMaxTxID) {
- if (encounteredError) {
- return Future.exception(new WriteException(bkDistributedLogManager.getStreamName(),
- "writer has been closed due to error."));
- }
- Future<BKLogSegmentWriter> writerFuture = asyncGetLedgerWriter(!disableRollOnSegmentError);
- if (null == writerFuture) {
- return rollLogSegmentIfNecessary(null, firstTxid, bestEffort, allowMaxTxID);
- } else if (rollLog) {
- return writerFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<BKLogSegmentWriter>>() {
- @Override
- public Future<BKLogSegmentWriter> apply(BKLogSegmentWriter writer) {
- return rollLogSegmentIfNecessary(writer, firstTxid, bestEffort, allowMaxTxID);
- }
- });
- } else {
- return writerFuture;
- }
- }
-
- /**
- * We write end of stream marker by writing a record with MAX_TXID, so we need to allow using
- * max txid when rolling for this case only.
- */
- private Future<BKLogSegmentWriter> getLogSegmentWriterForEndOfStream() {
- return getLogSegmentWriter(DistributedLogConstants.MAX_TXID,
- false /* bestEffort */,
- false /* roll log */,
- true /* allow max txid */);
- }
-
- private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
- boolean bestEffort,
- boolean rollLog) {
- return getLogSegmentWriter(firstTxid, bestEffort, rollLog, false /* allow max txid */);
- }
-
- Future<DLSN> queueRequest(LogRecord record, boolean flush) {
- PendingLogRecord pendingLogRecord = new PendingLogRecord(record, flush);
- pendingRequests.add(pendingLogRecord);
- return pendingLogRecord.promise;
- }
-
- boolean shouldRollLog(BKLogSegmentWriter w) {
- try {
- return null == w ||
- (!disableLogSegmentRollingFeature.isAvailable() &&
- shouldStartNewSegment(w));
- } catch (IOException ioe) {
- return false;
- }
- }
-
- void startQueueingRequests() {
- assert(null == pendingRequests && null == rollingFuture);
- pendingRequests = new LinkedList<PendingLogRecord>();
- rollingFuture = new Promise<BKLogSegmentWriter>();
- }
-
- // for ordering guarantee, we shouldn't send requests to next log segments until
- // previous log segment is done.
- private synchronized Future<DLSN> asyncWrite(final LogRecord record,
- boolean flush) {
- // The passed in writer may be stale since we acquire the writer outside of sync
- // lock. If we recently rolled and the new writer is cached, use that instead.
- Future<DLSN> result = null;
- BKLogSegmentWriter w;
- try {
- w = getCachedLogSegmentWriter();
- } catch (WriteException we) {
- return Future.exception(we);
- }
- if (null != rollingFuture) {
- if (streamFailFast) {
- result = Future.exception(new StreamNotReadyException("Rolling log segment"));
- } else {
- result = queueRequest(record, flush);
- }
- } else if (shouldRollLog(w)) {
- // insert a last record, so when it called back, we will trigger a log segment rolling
- startQueueingRequests();
- if (null != w) {
- LastPendingLogRecord lastLogRecordInCurrentSegment = new LastPendingLogRecord(record, flush);
- w.asyncWrite(record, true).addEventListener(lastLogRecordInCurrentSegment);
- result = lastLogRecordInCurrentSegment.promise;
- } else { // no log segment yet. roll the log segment and issue pending requests.
- result = queueRequest(record, flush);
- rollLogSegmentAndIssuePendingRequests(record.getTransactionId());
- }
- } else {
- result = w.asyncWrite(record, flush);
- }
- // use map here rather than onSuccess because we want lastTxId to be updated before
- // satisfying the future
- return result.map(new AbstractFunction1<DLSN, DLSN>() {
- @Override
- public DLSN apply(DLSN dlsn) {
- setLastTxId(record.getTransactionId());
- return dlsn;
- }
- });
- }
-
- private List<Future<DLSN>> asyncWriteBulk(List<LogRecord> records) {
- final ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(records.size());
- Iterator<LogRecord> iterator = records.iterator();
- while (iterator.hasNext()) {
- LogRecord record = iterator.next();
- Future<DLSN> future = asyncWrite(record, !iterator.hasNext());
- results.add(future);
-
- // Abort early if an individual write has already failed.
- Option<Try<DLSN>> result = future.poll();
- if (result.isDefined() && result.get().isThrow()) {
- break;
- }
- }
- if (records.size() > results.size()) {
- appendCancelledFutures(results, records.size() - results.size());
- }
- return results;
- }
-
- private void appendCancelledFutures(List<Future<DLSN>> futures, int numToAdd) {
- final WriteCancelledException cre =
- new WriteCancelledException(getStreamName());
- for (int i = 0; i < numToAdd; i++) {
- Future<DLSN> cancelledFuture = Future.exception(cre);
- futures.add(cancelledFuture);
- }
- }
-
- private void rollLogSegmentAndIssuePendingRequests(final long firstTxId) {
- getLogSegmentWriter(firstTxId, true, true)
- .addEventListener(new FutureEventListener<BKLogSegmentWriter>() {
- @Override
- public void onSuccess(BKLogSegmentWriter writer) {
- try {
- synchronized (BKAsyncLogWriter.this) {
- for (PendingLogRecord pendingLogRecord : pendingRequests) {
- FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_LogWriterIssuePending);
- writer.asyncWrite(pendingLogRecord.record, pendingLogRecord.flush)
- .addEventListener(pendingLogRecord);
- }
- // if there are no records in the pending queue, let's write a control record
- // so that when a new log segment is rolled, a control record will be added and
- // the corresponding bookies would be able to create its ledger.
- if (pendingRequests.isEmpty()) {
- LogRecord controlRecord = new LogRecord(firstTxId,
- DistributedLogConstants.CONTROL_RECORD_CONTENT);
- controlRecord.setControl();
- PendingLogRecord controlReq = new PendingLogRecord(controlRecord, false);
- writer.asyncWrite(controlReq.record, controlReq.flush)
- .addEventListener(controlReq);
- }
- if (null != rollingFuture) {
- FutureUtils.setValue(rollingFuture, writer);
- }
- rollingFuture = null;
- pendingRequestDispatch.add(pendingRequests.size());
- pendingRequests = null;
- }
- } catch (IOException ioe) {
- errorOutPendingRequestsAndWriter(ioe);
- }
- }
- @Override
- public void onFailure(Throwable cause) {
- errorOutPendingRequestsAndWriter(cause);
- }
- });
- }
-
- @VisibleForTesting
- void errorOutPendingRequests(Throwable cause, boolean errorOutWriter) {
- final List<PendingLogRecord> pendingRequestsSnapshot;
- synchronized (this) {
- pendingRequestsSnapshot = pendingRequests;
- encounteredError = errorOutWriter;
- pendingRequests = null;
- if (null != rollingFuture) {
- FutureUtils.setException(rollingFuture, cause);
- }
- rollingFuture = null;
- }
-
- pendingRequestDispatch.add(pendingRequestsSnapshot.size());
-
- // After erroring out the writer above, no more requests
- // will be enqueued to pendingRequests
- for (PendingLogRecord pendingLogRecord : pendingRequestsSnapshot) {
- pendingLogRecord.promise.setException(cause);
- }
- }
-
- void errorOutPendingRequestsAndWriter(Throwable cause) {
- errorOutPendingRequests(cause, true /* error out writer */);
- }
-
- /**
- * Write a log record to the stream.
- *
- * @param record single log record
- */
- @Override
- public Future<DLSN> write(final LogRecord record) {
- final Stopwatch stopwatch = Stopwatch.createStarted();
- return FutureUtils.stats(
- asyncWrite(record, true),
- writeOpStatsLogger,
- stopwatch);
- }
-
- /**
- * Write many log records to the stream. The return type here is unfortunate but its a direct result
- * of having to combine FuturePool and the asyncWriteBulk method which returns a future as well. The
- * problem is the List that asyncWriteBulk returns can't be materialized until getLogSegmentWriter
- * completes, so it has to be wrapped in a future itself.
- *
- * @param records list of records
- */
- @Override
- public Future<List<Future<DLSN>>> writeBulk(final List<LogRecord> records) {
- final Stopwatch stopwatch = Stopwatch.createStarted();
- return FutureUtils.stats(
- Future.value(asyncWriteBulk(records)),
- bulkWriteOpStatsLogger,
- stopwatch);
- }
-
- @Override
- public Future<Boolean> truncate(final DLSN dlsn) {
- if (DLSN.InvalidDLSN == dlsn) {
- return Future.value(false);
- }
- BKLogWriteHandler writeHandler;
- try {
- writeHandler = getWriteHandler();
- } catch (IOException e) {
- return Future.exception(e);
- }
- return writeHandler.setLogSegmentsOlderThanDLSNTruncated(dlsn).map(TruncationResultConverter);
- }
-
- Future<Long> flushAndCommit() {
- Future<BKLogSegmentWriter> writerFuture;
- synchronized (this) {
- if (null != this.rollingFuture) {
- writerFuture = this.rollingFuture;
- } else {
- writerFuture = getCachedLogWriterFuture();
- }
- }
- if (null == writerFuture) {
- return Future.value(getLastTxId());
- }
- return writerFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
- @Override
- public Future<Long> apply(BKLogSegmentWriter writer) {
- return writer.flushAndCommit();
- }
- });
- }
-
- Future<Long> markEndOfStream() {
- final Stopwatch stopwatch = Stopwatch.createStarted();
- Future<BKLogSegmentWriter> logSegmentWriterFuture;
- synchronized (this) {
- logSegmentWriterFuture = this.rollingFuture;
- }
- if (null == logSegmentWriterFuture) {
- logSegmentWriterFuture = getLogSegmentWriterForEndOfStream();
- }
-
- return FutureUtils.stats(
- logSegmentWriterFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
- @Override
- public Future<Long> apply(BKLogSegmentWriter w) {
- return w.markEndOfStream();
- }
- }),
- markEndOfStreamOpStatsLogger,
- stopwatch);
- }
-
- @Override
- protected Future<Void> asyncCloseAndComplete() {
- Future<BKLogSegmentWriter> logSegmentWriterFuture;
- synchronized (this) {
- logSegmentWriterFuture = this.rollingFuture;
- }
-
- if (null == logSegmentWriterFuture) {
- return super.asyncCloseAndComplete();
- } else {
- return logSegmentWriterFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Void>>() {
- @Override
- public Future<Void> apply(BKLogSegmentWriter segmentWriter) {
- return BKAsyncLogWriter.super.asyncCloseAndComplete();
- }
- });
- }
- }
-
- @Override
- void closeAndComplete() throws IOException {
- FutureUtils.result(asyncCloseAndComplete());
- }
-
- /**
- * *TEMP HACK*
- * Get the name of the stream this writer writes data to
- */
- @Override
- public String getStreamName() {
- return bkDistributedLogManager.getStreamName();
- }
-
- @Override
- public Future<Void> asyncAbort() {
- Future<Void> result = super.asyncAbort();
- synchronized (this) {
- if (pendingRequests != null) {
- for (PendingLogRecord pendingLogRecord : pendingRequests) {
- pendingLogRecord.promise.setException(new WriteException(bkDistributedLogManager.getStreamName(),
- "abort wring: writer has been closed due to error."));
- }
- }
- }
- return result;
- }
-
- @Override
- public String toString() {
- return String.format("AsyncLogWriter:%s", getStreamName());
- }
-}
[08/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
new file mode 100644
index 0000000..fdb29f3
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
@@ -0,0 +1,1325 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
+import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.function.GetLastTxIdFunction;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
+import org.apache.distributedlog.metadata.LogMetadataForWriter;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentFilter;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
+import org.apache.distributedlog.logsegment.RollingPolicy;
+import org.apache.distributedlog.logsegment.SizeBasedRollingPolicy;
+import org.apache.distributedlog.logsegment.TimeBasedRollingPolicy;
+import org.apache.distributedlog.metadata.LogStreamMetadataStore;
+import org.apache.distributedlog.metadata.MetadataUpdater;
+import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
+import org.apache.distributedlog.util.Allocator;
+import org.apache.distributedlog.util.DLUtils;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.FutureUtils.FutureEventListenerRunnable;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.Transaction;
+import org.apache.distributedlog.util.PermitLimiter;
+import org.apache.distributedlog.util.Utils;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+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.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.distributedlog.impl.ZKLogSegmentFilters.WRITE_HANDLE_FILTER;
+
+/**
+ * Log Handler for Writers.
+ *
+ * <h3>Metrics</h3>
+ * All the metrics about log write handler are exposed under scope `segments`.
+ * <ul>
+ * <li> `segments`/open : opstats. latency characteristics on starting a new log segment.
+ * <li> `segments`/close : opstats. latency characteristics on completing an inprogress log segment.
+ * <li> `segments`/recover : opstats. latency characteristics on recovering a log segment.
+ * <li> `segments`/delete : opstats. latency characteristics on deleting a log segment.
+ * </ul>
+ */
+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 MaxTxId maxTxId;
+ protected final MaxLogSegmentSequenceNo maxLogSegmentSequenceNo;
+ protected final boolean validateLogSegmentSequenceNumber;
+ protected final int regionId;
+ protected final RollingPolicy rollingPolicy;
+ protected Future<? extends DistributedLock> lockFuture = null;
+ protected final PermitLimiter writeLimiter;
+ protected final FeatureProvider featureProvider;
+ protected final DynamicDistributedLogConfiguration dynConf;
+ protected final MetadataUpdater metadataUpdater;
+ // tracking the inprogress log segments
+ protected final LinkedList<Long> inprogressLSSNs;
+
+ // Fetch LogSegments State: write can continue without full list of log segments while truncation needs
+ private final Future<Versioned<List<LogSegmentMetadata>>> fetchForWrite;
+ private Future<Versioned<List<LogSegmentMetadata>>> fetchForTruncation;
+
+ // Recover Functions
+ private final RecoverLogSegmentFunction recoverLogSegmentFunction =
+ new RecoverLogSegmentFunction();
+ private final AbstractFunction1<List<LogSegmentMetadata>, Future<Long>> recoverLogSegmentsFunction =
+ new AbstractFunction1<List<LogSegmentMetadata>, Future<Long>>() {
+ @Override
+ public Future<Long> apply(List<LogSegmentMetadata> segmentList) {
+ LOG.info("Initiating Recovery For {} : {}", getFullyQualifiedName(), segmentList);
+ // if lastLedgerRollingTimeMillis is not updated, we set it to now.
+ synchronized (BKLogWriteHandler.this) {
+ if (lastLedgerRollingTimeMillis < 0) {
+ lastLedgerRollingTimeMillis = Utils.nowInMillis();
+ }
+ }
+
+ if (validateLogSegmentSequenceNumber) {
+ synchronized (inprogressLSSNs) {
+ for (LogSegmentMetadata segment : segmentList) {
+ if (segment.isInProgress()) {
+ inprogressLSSNs.addLast(segment.getLogSegmentSequenceNumber());
+ }
+ }
+ }
+ }
+
+ return FutureUtils.processList(segmentList, recoverLogSegmentFunction, scheduler).map(
+ GetLastTxIdFunction.INSTANCE);
+ }
+ };
+
+ // Stats
+ private final StatsLogger perLogStatsLogger;
+ private final OpStatsLogger closeOpStats;
+ private final OpStatsLogger openOpStats;
+ private final OpStatsLogger recoverOpStats;
+ private final OpStatsLogger deleteOpStats;
+
+ /**
+ * Construct a Bookkeeper journal manager.
+ */
+ BKLogWriteHandler(LogMetadataForWriter logMetadata,
+ DistributedLogConfiguration conf,
+ LogStreamMetadataStore streamMetadataStore,
+ LogSegmentMetadataCache metadataCache,
+ LogSegmentEntryStore entryStore,
+ OrderedScheduler scheduler,
+ Allocator<LogSegmentEntryWriter, Object> segmentAllocator,
+ StatsLogger statsLogger,
+ StatsLogger perLogStatsLogger,
+ AlertStatsLogger alertStatsLogger,
+ String clientId,
+ int regionId,
+ PermitLimiter writeLimiter,
+ FeatureProvider featureProvider,
+ DynamicDistributedLogConfiguration dynConf,
+ DistributedLock lock /** owned by handler **/) {
+ super(logMetadata,
+ conf,
+ streamMetadataStore,
+ metadataCache,
+ entryStore,
+ scheduler,
+ statsLogger,
+ alertStatsLogger,
+ clientId);
+ this.logMetadataForWriter = logMetadata;
+ this.logSegmentAllocator = segmentAllocator;
+ this.perLogStatsLogger = perLogStatsLogger;
+ this.writeLimiter = writeLimiter;
+ this.featureProvider = featureProvider;
+ this.dynConf = dynConf;
+ this.lock = lock;
+ this.metadataUpdater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
+
+ if (conf.getEncodeRegionIDInLogSegmentMetadata()) {
+ this.regionId = regionId;
+ } else {
+ this.regionId = DistributedLogConstants.LOCAL_REGION_ID;
+ }
+ this.validateLogSegmentSequenceNumber = conf.isLogSegmentSequenceNumberValidationEnabled();
+
+ // Construct the max sequence no
+ maxLogSegmentSequenceNo = new MaxLogSegmentSequenceNo(logMetadata.getMaxLSSNData());
+ inprogressLSSNs = new LinkedList<Long>();
+ // Construct the max txn id.
+ maxTxId = new MaxTxId(logMetadata.getMaxTxIdData());
+
+ // Schedule fetching log segment list in background before we access it.
+ // We don't need to watch the log segment list changes for writer, as it manages log segment list.
+ fetchForWrite = readLogSegmentsFromStore(
+ LogSegmentMetadata.COMPARATOR,
+ WRITE_HANDLE_FILTER,
+ null);
+
+ // Initialize other parameters.
+ setLastLedgerRollingTimeMillis(Utils.nowInMillis());
+
+ // Rolling Policy
+ if (conf.getLogSegmentRollingIntervalMinutes() > 0) {
+ rollingPolicy = new TimeBasedRollingPolicy(conf.getLogSegmentRollingIntervalMinutes() * 60 * 1000L);
+ } else {
+ rollingPolicy = new SizeBasedRollingPolicy(conf.getMaxLogSegmentBytes());
+ }
+
+ // Stats
+ StatsLogger segmentsStatsLogger = statsLogger.scope("segments");
+ openOpStats = segmentsStatsLogger.getOpStatsLogger("open");
+ closeOpStats = segmentsStatsLogger.getOpStatsLogger("close");
+ recoverOpStats = segmentsStatsLogger.getOpStatsLogger("recover");
+ deleteOpStats = segmentsStatsLogger.getOpStatsLogger("delete");
+ }
+
+ private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFetch(
+ final Comparator<LogSegmentMetadata> comparator) {
+ final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
+ fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ FutureUtils.setException(promise, cause);
+ }
+
+ @Override
+ public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
+ try {
+ FutureUtils.setValue(promise, getCachedLogSegments(comparator));
+ } catch (UnexpectedException e) {
+ FutureUtils.setException(promise, e);
+ }
+ }
+ });
+ return promise;
+ }
+
+ private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFullFetch(
+ final Comparator<LogSegmentMetadata> comparator) {
+ Future<Versioned<List<LogSegmentMetadata>>> result;
+ synchronized (this) {
+ if (null == fetchForTruncation) {
+ fetchForTruncation = readLogSegmentsFromStore(
+ LogSegmentMetadata.COMPARATOR,
+ LogSegmentFilter.DEFAULT_FILTER,
+ null);
+ }
+ result = fetchForTruncation;
+ }
+
+ final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
+ result.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ FutureUtils.setException(promise, cause);
+ }
+
+ @Override
+ public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
+ try {
+ FutureUtils.setValue(promise, getCachedLogSegments(comparator));
+ } catch (UnexpectedException e) {
+ FutureUtils.setException(promise, e);
+ }
+ }
+ });
+ return promise;
+ }
+
+ // Transactional operations for MaxLogSegmentSequenceNo
+ void storeMaxSequenceNumber(final Transaction<Object> txn,
+ final MaxLogSegmentSequenceNo maxSeqNo,
+ final long seqNo,
+ final boolean isInprogress) {
+ metadataStore.storeMaxLogSegmentSequenceNumber(txn, logMetadata, maxSeqNo.getVersionedData(seqNo),
+ new Transaction.OpListener<Version>() {
+ @Override
+ public void onCommit(Version version) {
+ if (validateLogSegmentSequenceNumber) {
+ synchronized (inprogressLSSNs) {
+ if (isInprogress) {
+ inprogressLSSNs.add(seqNo);
+ } else {
+ inprogressLSSNs.removeFirst();
+ }
+ }
+ }
+ maxSeqNo.update(version, seqNo);
+ }
+
+ @Override
+ public void onAbort(Throwable t) {
+ // no-op
+ }
+ });
+ }
+
+ // Transactional operations for MaxTxId
+ void storeMaxTxId(final Transaction<Object> txn,
+ final MaxTxId maxTxId,
+ final long txId) {
+ metadataStore.storeMaxTxnId(txn, logMetadataForWriter, maxTxId.getVersionedData(txId),
+ new Transaction.OpListener<Version>() {
+ @Override
+ public void onCommit(Version version) {
+ maxTxId.update(version, txId);
+ }
+
+ @Override
+ public void onAbort(Throwable t) {
+ // no-op
+ }
+ });
+ }
+
+ // Transactional operations for logsegment
+ void writeLogSegment(final Transaction<Object> txn,
+ final LogSegmentMetadata metadata) {
+ metadataStore.createLogSegment(txn, metadata, new Transaction.OpListener<Void>() {
+ @Override
+ public void onCommit(Void r) {
+ addLogSegmentToCache(metadata.getSegmentName(), metadata);
+ }
+
+ @Override
+ public void onAbort(Throwable t) {
+ // no-op
+ }
+ });
+ }
+
+ void deleteLogSegment(final Transaction<Object> txn,
+ final LogSegmentMetadata metadata) {
+ metadataStore.deleteLogSegment(txn, metadata, new Transaction.OpListener<Void>() {
+ @Override
+ public void onCommit(Void r) {
+ removeLogSegmentFromCache(metadata.getSegmentName());
+ }
+
+ @Override
+ public void onAbort(Throwable t) {
+ // no-op
+ }
+ });
+ }
+
+ /**
+ * The caller could call this before any actions, which to hold the lock for
+ * the write handler of its whole lifecycle. The lock will only be released
+ * when closing the write handler.
+ *
+ * This method is useful to prevent releasing underlying zookeeper lock during
+ * recovering/completing log segments. Releasing underlying zookeeper lock means
+ * 1) increase latency when re-lock on starting new log segment. 2) increase the
+ * possibility of a stream being re-acquired by other instances.
+ *
+ * @return future represents the lock result
+ */
+ Future<? extends DistributedLock> lockHandler() {
+ if (null != lockFuture) {
+ return lockFuture;
+ }
+ lockFuture = lock.asyncAcquire();
+ return lockFuture;
+ }
+
+ Future<Void> unlockHandler() {
+ if (null != lockFuture) {
+ return lock.asyncClose();
+ } else {
+ return Future.Void();
+ }
+ }
+
+ /**
+ * Start a new log segment in a BookKeeper ledger.
+ * First ensure that we have the write lock for this journal.
+ * Then create a ledger and stream based on that ledger.
+ * The ledger id is written to the inprogress znode, so that in the
+ * case of a crash, a recovery process can find the ledger we were writing
+ * to when we crashed.
+ *
+ * @param txId First transaction id to be written to the stream
+ * @return
+ * @throws IOException
+ */
+ public BKLogSegmentWriter startLogSegment(long txId) throws IOException {
+ return startLogSegment(txId, false, false);
+ }
+
+ /**
+ * Start a new log segment in a BookKeeper ledger.
+ * First ensure that we have the write lock for this journal.
+ * Then create a ledger and stream based on that ledger.
+ * The ledger id is written to the inprogress znode, so that in the
+ * case of a crash, a recovery process can find the ledger we were writing
+ * to when we crashed.
+ *
+ * @param txId First transaction id to be written to the stream
+ * @param bestEffort
+ * @param allowMaxTxID
+ * allow using max tx id to start log segment
+ * @return
+ * @throws IOException
+ */
+ public BKLogSegmentWriter startLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID)
+ throws IOException {
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ boolean success = false;
+ try {
+ BKLogSegmentWriter writer = doStartLogSegment(txId, bestEffort, allowMaxTxID);
+ success = true;
+ return writer;
+ } finally {
+ if (success) {
+ openOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ } else {
+ openOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ }
+ }
+ }
+
+ protected long assignLogSegmentSequenceNumber() throws IOException {
+ // For any active stream we will always make sure that there is at least one
+ // active ledger (except when the stream first starts out). Therefore when we
+ // see no ledger metadata for a stream, we assume that this is the first ledger
+ // in the stream
+ long logSegmentSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
+ boolean logSegmentsFound = false;
+
+ if (LogSegmentMetadata.supportsLogSegmentSequenceNo(conf.getDLLedgerMetadataLayoutVersion())) {
+ List<LogSegmentMetadata> ledgerListDesc = getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR);
+ Long nextLogSegmentSeqNo = DLUtils.nextLogSegmentSequenceNumber(ledgerListDesc);
+
+ if (null == nextLogSegmentSeqNo) {
+ logSegmentsFound = false;
+ // we don't find last assigned log segment sequence number
+ // then we start the log segment with configured FirstLogSegmentSequenceNumber.
+ logSegmentSeqNo = conf.getFirstLogSegmentSequenceNumber();
+ } else {
+ logSegmentsFound = true;
+ // latest log segment is assigned with a sequence number, start with next sequence number
+ logSegmentSeqNo = nextLogSegmentSeqNo;
+ }
+ }
+
+ // We only skip log segment sequence number validation only when no log segments found &
+ // the maximum log segment sequence number is "UNASSIGNED".
+ if (!logSegmentsFound &&
+ (DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO == maxLogSegmentSequenceNo.getSequenceNumber())) {
+ // no ledger seqno stored in /ledgers before
+ LOG.info("No max ledger sequence number found while creating log segment {} for {}.",
+ logSegmentSeqNo, getFullyQualifiedName());
+ } else if (maxLogSegmentSequenceNo.getSequenceNumber() + 1 != logSegmentSeqNo) {
+ LOG.warn("Unexpected max log segment sequence number {} for {} : list of cached segments = {}",
+ new Object[]{maxLogSegmentSequenceNo.getSequenceNumber(), getFullyQualifiedName(),
+ getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)});
+ // there is max log segment number recorded there and it isn't match. throw exception.
+ throw new DLIllegalStateException("Unexpected max log segment sequence number "
+ + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
+ + ", expected " + (logSegmentSeqNo - 1));
+ }
+
+ return logSegmentSeqNo;
+ }
+
+ protected BKLogSegmentWriter doStartLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID) throws IOException {
+ return FutureUtils.result(asyncStartLogSegment(txId, bestEffort, allowMaxTxID));
+ }
+
+ protected Future<BKLogSegmentWriter> asyncStartLogSegment(final long txId,
+ final boolean bestEffort,
+ final boolean allowMaxTxID) {
+ final Promise<BKLogSegmentWriter> promise = new Promise<BKLogSegmentWriter>();
+ try {
+ lock.checkOwnershipAndReacquire();
+ } catch (LockingException e) {
+ FutureUtils.setException(promise, e);
+ return promise;
+ }
+ fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ FutureUtils.setException(promise, cause);
+ }
+
+ @Override
+ public void onSuccess(Versioned<List<LogSegmentMetadata>> list) {
+ doStartLogSegment(txId, bestEffort, allowMaxTxID, promise);
+ }
+ });
+ return promise;
+ }
+
+ protected void doStartLogSegment(final long txId,
+ final boolean bestEffort,
+ final boolean allowMaxTxID,
+ final Promise<BKLogSegmentWriter> promise) {
+ // validate the tx id
+ if ((txId < 0) ||
+ (!allowMaxTxID && (txId == DistributedLogConstants.MAX_TXID))) {
+ FutureUtils.setException(promise, new IOException("Invalid Transaction Id " + txId));
+ return;
+ }
+
+ long highestTxIdWritten = maxTxId.get();
+ if (txId < highestTxIdWritten) {
+ if (highestTxIdWritten == DistributedLogConstants.MAX_TXID) {
+ LOG.error("We've already marked the stream as ended and attempting to start a new log segment");
+ FutureUtils.setException(promise, new EndOfStreamException("Writing to a stream after it has been marked as completed"));
+ return;
+ } else {
+ LOG.error("We've already seen TxId {} the max TXId is {}", txId, highestTxIdWritten);
+ FutureUtils.setException(promise, new TransactionIdOutOfOrderException(txId, highestTxIdWritten));
+ return;
+ }
+ }
+
+ try {
+ logSegmentAllocator.allocate();
+ } catch (IOException e) {
+ // failed to issue an allocation request
+ failStartLogSegment(promise, bestEffort, e);
+ return;
+ }
+
+ // start the transaction from zookeeper
+ final Transaction<Object> txn = streamMetadataStore.newTransaction();
+
+ // failpoint injected before creating ledger
+ try {
+ FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_StartLogSegmentBeforeLedgerCreate);
+ } catch (IOException ioe) {
+ failStartLogSegment(promise, bestEffort, ioe);
+ return;
+ }
+
+ logSegmentAllocator.tryObtain(txn, NULL_OP_LISTENER)
+ .addEventListener(new FutureEventListener<LogSegmentEntryWriter>() {
+
+ @Override
+ public void onSuccess(LogSegmentEntryWriter entryWriter) {
+ // try-obtain succeed
+ createInprogressLogSegment(
+ txn,
+ txId,
+ entryWriter,
+ bestEffort,
+ promise);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ failStartLogSegment(promise, bestEffort, cause);
+ }
+ });
+ }
+
+ private void failStartLogSegment(Promise<BKLogSegmentWriter> promise,
+ boolean bestEffort,
+ Throwable cause) {
+ if (bestEffort) {
+ FutureUtils.setValue(promise, null);
+ } else {
+ FutureUtils.setException(promise, cause);
+ }
+ }
+
+ // once the ledger handle is obtained from allocator, this function should guarantee
+ // either the transaction is executed or aborted. Otherwise, the ledger handle will
+ // just leak from the allocation pool - hence cause "No Ledger Allocator"
+ private void createInprogressLogSegment(Transaction<Object> txn,
+ final long txId,
+ final LogSegmentEntryWriter entryWriter,
+ boolean bestEffort,
+ final Promise<BKLogSegmentWriter> promise) {
+ final long logSegmentSeqNo;
+ try {
+ FailpointUtils.checkFailPoint(
+ FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber);
+ logSegmentSeqNo = assignLogSegmentSequenceNumber();
+ } catch (IOException e) {
+ // abort the current prepared transaction
+ txn.abort(e);
+ failStartLogSegment(promise, bestEffort, e);
+ return;
+ }
+
+ final String inprogressZnodePath = inprogressZNode(
+ entryWriter.getLogSegmentId(), txId, logSegmentSeqNo);
+ final LogSegmentMetadata l =
+ new LogSegmentMetadata.LogSegmentMetadataBuilder(inprogressZnodePath,
+ conf.getDLLedgerMetadataLayoutVersion(), entryWriter.getLogSegmentId(), txId)
+ .setLogSegmentSequenceNo(logSegmentSeqNo)
+ .setRegionId(regionId)
+ .setEnvelopeEntries(
+ LogSegmentMetadata.supportsEnvelopedEntries(conf.getDLLedgerMetadataLayoutVersion()))
+ .build();
+
+ // Create an inprogress segment
+ writeLogSegment(txn, l);
+
+ // Try storing max sequence number.
+ LOG.debug("Try storing max sequence number in startLogSegment {} : {}", inprogressZnodePath, logSegmentSeqNo);
+ storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, logSegmentSeqNo, true);
+
+ // Try storing max tx id.
+ LOG.debug("Try storing MaxTxId in startLogSegment {} {}", inprogressZnodePath, txId);
+ storeMaxTxId(txn, maxTxId, txId);
+
+ txn.execute().addEventListener(FutureEventListenerRunnable.of(new FutureEventListener<Void>() {
+
+ @Override
+ public void onSuccess(Void value) {
+ try {
+ FutureUtils.setValue(promise, new BKLogSegmentWriter(
+ getFullyQualifiedName(),
+ l.getSegmentName(),
+ conf,
+ conf.getDLLedgerMetadataLayoutVersion(),
+ entryWriter,
+ lock,
+ txId,
+ logSegmentSeqNo,
+ scheduler,
+ statsLogger,
+ perLogStatsLogger,
+ alertStatsLogger,
+ writeLimiter,
+ featureProvider,
+ dynConf));
+ } catch (IOException ioe) {
+ failStartLogSegment(promise, false, ioe);
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ failStartLogSegment(promise, false, cause);
+ }
+ }, scheduler));
+ }
+
+ boolean shouldStartNewSegment(BKLogSegmentWriter writer) {
+ return rollingPolicy.shouldRollover(writer, lastLedgerRollingTimeMillis);
+ }
+
+ /**
+ * Finalize a log segment. If the journal manager is currently
+ * writing to a ledger, ensure that this is the ledger of the log segment
+ * being finalized.
+ * <p/>
+ * Otherwise this is the recovery case. In the recovery case, ensure that
+ * the firstTxId of the ledger matches firstTxId for the segment we are
+ * trying to finalize.
+ */
+ Future<LogSegmentMetadata> completeAndCloseLogSegment(final BKLogSegmentWriter writer) {
+ final Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+ completeAndCloseLogSegment(writer, promise);
+ return promise;
+ }
+
+ private void completeAndCloseLogSegment(final BKLogSegmentWriter writer,
+ final Promise<LogSegmentMetadata> promise) {
+ writer.asyncClose().addEventListener(new FutureEventListener<Void>() {
+ @Override
+ public void onSuccess(Void value) {
+ // in theory closeToFinalize should throw exception if a stream is in error.
+ // just in case, add another checking here to make sure we don't close log segment is a stream is in error.
+ if (writer.shouldFailCompleteLogSegment()) {
+ FutureUtils.setException(promise,
+ new IOException("LogSegmentWriter for " + writer.getFullyQualifiedLogSegment() + " is already in error."));
+ return;
+ }
+ doCompleteAndCloseLogSegment(
+ inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
+ writer.getLogSegmentSequenceNumber(),
+ writer.getLogSegmentId(),
+ writer.getStartTxId(),
+ writer.getLastTxId(),
+ writer.getPositionWithinLogSegment(),
+ writer.getLastDLSN().getEntryId(),
+ writer.getLastDLSN().getSlotId(),
+ promise);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ FutureUtils.setException(promise, cause);
+ }
+ });
+ }
+
+ @VisibleForTesting
+ LogSegmentMetadata completeAndCloseLogSegment(long logSegmentSeqNo,
+ long logSegmentId,
+ long firstTxId,
+ long lastTxId,
+ int recordCount)
+ throws IOException {
+ return completeAndCloseLogSegment(inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo), logSegmentSeqNo,
+ logSegmentId, firstTxId, lastTxId, recordCount, -1, -1);
+ }
+
+ /**
+ * Finalize a log segment. If the journal manager is currently
+ * writing to a ledger, ensure that this is the ledger of the log segment
+ * being finalized.
+ * <p/>
+ * Otherwise this is the recovery case. In the recovery case, ensure that
+ * the firstTxId of the ledger matches firstTxId for the segment we are
+ * trying to finalize.
+ */
+ LogSegmentMetadata completeAndCloseLogSegment(String inprogressZnodeName, long logSegmentSeqNo,
+ long logSegmentId, long firstTxId, long lastTxId,
+ int recordCount, long lastEntryId, long lastSlotId)
+ throws IOException {
+ Stopwatch stopwatch = Stopwatch.createStarted();
+ boolean success = false;
+ try {
+ LogSegmentMetadata completedLogSegment =
+ doCompleteAndCloseLogSegment(inprogressZnodeName, logSegmentSeqNo,
+ logSegmentId, firstTxId, lastTxId, recordCount,
+ lastEntryId, lastSlotId);
+ success = true;
+ return completedLogSegment;
+ } finally {
+ if (success) {
+ closeOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ } else {
+ closeOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ }
+ }
+ }
+
+ protected long computeStartSequenceId(LogSegmentMetadata segment) throws IOException {
+ if (!segment.isInProgress()) {
+ return segment.getStartSequenceId();
+ }
+
+ long startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
+
+ // we only record sequence id when both write version and logsegment's version support sequence id
+ if (LogSegmentMetadata.supportsSequenceId(conf.getDLLedgerMetadataLayoutVersion())
+ && segment.supportsSequenceId()) {
+ List<LogSegmentMetadata> logSegmentDescList =
+ getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR);
+ startSequenceId = DLUtils.computeStartSequenceId(logSegmentDescList, segment);
+ }
+
+ return startSequenceId;
+ }
+
+ /**
+ * Close log segment
+ *
+ * @param inprogressZnodeName
+ * @param logSegmentSeqNo
+ * @param logSegmentId
+ * @param firstTxId
+ * @param lastTxId
+ * @param recordCount
+ * @param lastEntryId
+ * @param lastSlotId
+ * @throws IOException
+ */
+ protected LogSegmentMetadata doCompleteAndCloseLogSegment(
+ String inprogressZnodeName,
+ long logSegmentSeqNo,
+ long logSegmentId,
+ long firstTxId,
+ long lastTxId,
+ int recordCount,
+ long lastEntryId,
+ long lastSlotId) throws IOException {
+ Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+ doCompleteAndCloseLogSegment(
+ inprogressZnodeName,
+ logSegmentSeqNo,
+ logSegmentId,
+ firstTxId,
+ lastTxId,
+ recordCount,
+ lastEntryId,
+ lastSlotId,
+ promise);
+ return FutureUtils.result(promise);
+ }
+
+ protected void doCompleteAndCloseLogSegment(final String inprogressZnodeName,
+ final long logSegmentSeqNo,
+ final long logSegmentId,
+ final long firstTxId,
+ final long lastTxId,
+ final int recordCount,
+ final long lastEntryId,
+ final long lastSlotId,
+ final Promise<LogSegmentMetadata> promise) {
+ fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ FutureUtils.setException(promise, cause);
+ }
+
+ @Override
+ public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
+ doCompleteAndCloseLogSegmentAfterLogSegmentListFetched(
+ inprogressZnodeName,
+ logSegmentSeqNo,
+ logSegmentId,
+ firstTxId,
+ lastTxId,
+ recordCount,
+ lastEntryId,
+ lastSlotId,
+ promise);
+ }
+ });
+ }
+
+ private void doCompleteAndCloseLogSegmentAfterLogSegmentListFetched(
+ final String inprogressZnodeName,
+ long logSegmentSeqNo,
+ long logSegmentId,
+ long firstTxId,
+ long lastTxId,
+ int recordCount,
+ long lastEntryId,
+ long lastSlotId,
+ final Promise<LogSegmentMetadata> promise) {
+ try {
+ lock.checkOwnershipAndReacquire();
+ } catch (IOException ioe) {
+ FutureUtils.setException(promise, ioe);
+ return;
+ }
+
+ LOG.debug("Completing and Closing Log Segment {} {}", firstTxId, lastTxId);
+ LogSegmentMetadata inprogressLogSegment = readLogSegmentFromCache(inprogressZnodeName);
+
+ // validate log segment
+ if (inprogressLogSegment.getLogSegmentId() != logSegmentId) {
+ FutureUtils.setException(promise, new IOException(
+ "Active ledger has different ID to inprogress. "
+ + inprogressLogSegment.getLogSegmentId() + " found, "
+ + logSegmentId + " expected"));
+ return;
+ }
+ // validate the transaction id
+ if (inprogressLogSegment.getFirstTxId() != firstTxId) {
+ FutureUtils.setException(promise, new IOException("Transaction id not as expected, "
+ + inprogressLogSegment.getFirstTxId() + " found, " + firstTxId + " expected"));
+ return;
+ }
+ // validate the log sequence number
+ if (validateLogSegmentSequenceNumber) {
+ synchronized (inprogressLSSNs) {
+ if (inprogressLSSNs.isEmpty()) {
+ FutureUtils.setException(promise, new UnexpectedException(
+ "Didn't find matched inprogress log segments when completing inprogress "
+ + inprogressLogSegment));
+ return;
+ }
+ long leastInprogressLSSN = inprogressLSSNs.getFirst();
+ // the log segment sequence number in metadata {@link inprogressLogSegment.getLogSegmentSequenceNumber()}
+ // should be same as the sequence number we are completing (logSegmentSeqNo)
+ // and
+ // it should also be same as the least inprogress log segment sequence number tracked in {@link inprogressLSSNs}
+ if ((inprogressLogSegment.getLogSegmentSequenceNumber() != logSegmentSeqNo) ||
+ (leastInprogressLSSN != logSegmentSeqNo)) {
+ FutureUtils.setException(promise, new UnexpectedException(
+ "Didn't find matched inprogress log segments when completing inprogress "
+ + inprogressLogSegment));
+ return;
+ }
+ }
+ }
+
+ // store max sequence number.
+ long maxSeqNo= Math.max(logSegmentSeqNo, maxLogSegmentSequenceNo.getSequenceNumber());
+ if (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo ||
+ (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo + 1)) {
+ // ignore the case that a new inprogress log segment is pre-allocated
+ // before completing current inprogress one
+ LOG.info("Try storing max sequence number {} in completing {}.",
+ new Object[] { logSegmentSeqNo, inprogressLogSegment.getZkPath() });
+ } else {
+ LOG.warn("Unexpected max ledger sequence number {} found while completing log segment {} for {}",
+ new Object[] { maxLogSegmentSequenceNo.getSequenceNumber(), logSegmentSeqNo, getFullyQualifiedName() });
+ if (validateLogSegmentSequenceNumber) {
+ FutureUtils.setException(promise, new DLIllegalStateException("Unexpected max log segment sequence number "
+ + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
+ + ", expected " + (logSegmentSeqNo - 1)));
+ return;
+ }
+ }
+
+ // Prepare the completion
+ final String pathForCompletedLedger = completedLedgerZNode(firstTxId, lastTxId, logSegmentSeqNo);
+ long startSequenceId;
+ try {
+ startSequenceId = computeStartSequenceId(inprogressLogSegment);
+ } catch (IOException ioe) {
+ FutureUtils.setException(promise, ioe);
+ return;
+ }
+ // write completed ledger znode
+ final LogSegmentMetadata completedLogSegment =
+ inprogressLogSegment.completeLogSegment(
+ pathForCompletedLedger,
+ lastTxId,
+ recordCount,
+ lastEntryId,
+ lastSlotId,
+ startSequenceId);
+ setLastLedgerRollingTimeMillis(completedLogSegment.getCompletionTime());
+
+ // prepare the transaction
+ Transaction<Object> txn = streamMetadataStore.newTransaction();
+
+ // create completed log segment
+ writeLogSegment(txn, completedLogSegment);
+ // delete inprogress log segment
+ deleteLogSegment(txn, inprogressLogSegment);
+ // store max sequence number
+ storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, maxSeqNo, false);
+ // update max txn id.
+ LOG.debug("Trying storing LastTxId in Finalize Path {} LastTxId {}", pathForCompletedLedger, lastTxId);
+ storeMaxTxId(txn, maxTxId, lastTxId);
+
+ txn.execute().addEventListener(FutureEventListenerRunnable.of(new FutureEventListener<Void>() {
+ @Override
+ public void onSuccess(Void value) {
+ LOG.info("Completed {} to {} for {} : {}",
+ new Object[] { inprogressZnodeName, completedLogSegment.getSegmentName(),
+ getFullyQualifiedName(), completedLogSegment });
+ FutureUtils.setValue(promise, completedLogSegment);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ FutureUtils.setException(promise, cause);
+ }
+ }, scheduler));
+ }
+
+ public Future<Long> recoverIncompleteLogSegments() {
+ try {
+ FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_RecoverIncompleteLogSegments);
+ } catch (IOException ioe) {
+ return Future.exception(ioe);
+ }
+ return getCachedLogSegmentsAfterFirstFetch(LogSegmentMetadata.COMPARATOR).flatMap(recoverLogSegmentsFunction);
+ }
+
+ class RecoverLogSegmentFunction extends Function<LogSegmentMetadata, Future<LogSegmentMetadata>> {
+
+ @Override
+ public Future<LogSegmentMetadata> apply(final LogSegmentMetadata l) {
+ if (!l.isInProgress()) {
+ return Future.value(l);
+ }
+
+ LOG.info("Recovering last record in log segment {} for {}.", l, getFullyQualifiedName());
+ return asyncReadLastRecord(l, true, true, true).flatMap(
+ new AbstractFunction1<LogRecordWithDLSN, Future<LogSegmentMetadata>>() {
+ @Override
+ public Future<LogSegmentMetadata> apply(LogRecordWithDLSN lastRecord) {
+ return completeLogSegment(l, lastRecord);
+ }
+ });
+ }
+
+ private Future<LogSegmentMetadata> completeLogSegment(LogSegmentMetadata l,
+ LogRecordWithDLSN lastRecord) {
+ LOG.info("Recovered last record in log segment {} for {}.", l, getFullyQualifiedName());
+
+ long endTxId = DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID;
+ int recordCount = 0;
+ long lastEntryId = -1;
+ long lastSlotId = -1;
+
+ if (null != lastRecord) {
+ endTxId = lastRecord.getTransactionId();
+ recordCount = lastRecord.getLastPositionWithinLogSegment();
+ lastEntryId = lastRecord.getDlsn().getEntryId();
+ lastSlotId = lastRecord.getDlsn().getSlotId();
+ }
+
+ if (endTxId == DistributedLogConstants.INVALID_TXID) {
+ LOG.error("Unrecoverable corruption has occurred in segment "
+ + l.toString() + " at path " + l.getZkPath()
+ + ". Unable to continue recovery.");
+ return Future.exception(new IOException("Unrecoverable corruption,"
+ + " please check logs."));
+ } else if (endTxId == DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID) {
+ // TODO: Empty ledger - Ideally we should just remove it?
+ endTxId = l.getFirstTxId();
+ }
+
+ Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+ doCompleteAndCloseLogSegment(
+ l.getZNodeName(),
+ l.getLogSegmentSequenceNumber(),
+ l.getLogSegmentId(),
+ l.getFirstTxId(),
+ endTxId,
+ recordCount,
+ lastEntryId,
+ lastSlotId,
+ promise);
+ return promise;
+ }
+
+ }
+
+ Future<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(final DLSN dlsn) {
+ if (DLSN.InvalidDLSN == dlsn) {
+ List<LogSegmentMetadata> emptyList = new ArrayList<LogSegmentMetadata>(0);
+ return Future.value(emptyList);
+ }
+ return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
+ new AbstractFunction1<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
+ @Override
+ public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
+ return setLogSegmentsOlderThanDLSNTruncated(logSegments, dlsn);
+ }
+ });
+ }
+
+ private Future<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(List<LogSegmentMetadata> logSegments,
+ final DLSN dlsn) {
+ LOG.debug("Setting truncation status on logs older than {} from {} for {}",
+ new Object[]{dlsn, logSegments, getFullyQualifiedName()});
+ List<LogSegmentMetadata> truncateList = new ArrayList<LogSegmentMetadata>(logSegments.size());
+ LogSegmentMetadata partialTruncate = null;
+ LOG.info("{}: Truncating log segments older than {}", getFullyQualifiedName(), dlsn);
+ for (int i = 0; i < logSegments.size(); i++) {
+ LogSegmentMetadata l = logSegments.get(i);
+ if (!l.isInProgress()) {
+ if (l.getLastDLSN().compareTo(dlsn) < 0) {
+ LOG.debug("{}: Truncating log segment {} ", getFullyQualifiedName(), l);
+ truncateList.add(l);
+ } else if (l.getFirstDLSN().compareTo(dlsn) < 0) {
+ // Can be satisfied by at most one segment
+ if (null != partialTruncate) {
+ String logMsg = String.format("Potential metadata inconsistency for stream %s at segment %s", getFullyQualifiedName(), l);
+ LOG.error(logMsg);
+ return Future.exception(new DLIllegalStateException(logMsg));
+ }
+ LOG.info("{}: Partially truncating log segment {} older than {}.", new Object[] {getFullyQualifiedName(), l, dlsn});
+ partialTruncate = l;
+ } else {
+ break;
+ }
+ } else {
+ break;
+ }
+ }
+ return setLogSegmentTruncationStatus(truncateList, partialTruncate, dlsn);
+ }
+
+ private int getNumCandidateLogSegmentsToPurge(List<LogSegmentMetadata> logSegments) {
+ if (logSegments.isEmpty()) {
+ return 0;
+ } else {
+ // we have to keep at least one completed log segment for sequence id
+ int numCandidateLogSegments = 0;
+ for (LogSegmentMetadata segment : logSegments) {
+ if (segment.isInProgress()) {
+ break;
+ } else {
+ ++numCandidateLogSegments;
+ }
+ }
+
+ return numCandidateLogSegments - 1;
+ }
+ }
+
+ Future<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTimestamp(final long minTimestampToKeep) {
+ if (minTimestampToKeep >= Utils.nowInMillis()) {
+ return Future.exception(new IllegalArgumentException(
+ "Invalid timestamp " + minTimestampToKeep + " to purge logs for " + getFullyQualifiedName()));
+ }
+ return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
+ new Function<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
+ @Override
+ public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
+ List<LogSegmentMetadata> purgeList = new ArrayList<LogSegmentMetadata>(logSegments.size());
+
+ int numCandidates = getNumCandidateLogSegmentsToPurge(logSegments);
+
+ for (int iterator = 0; iterator < numCandidates; iterator++) {
+ LogSegmentMetadata l = logSegments.get(iterator);
+ // When application explicitly truncates segments; timestamp based purge is
+ // only used to cleanup log segments that have been marked for truncation
+ if ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) &&
+ !l.isInProgress() && (l.getCompletionTime() < minTimestampToKeep)) {
+ purgeList.add(l);
+ } else {
+ // stop truncating log segments if we find either an inprogress or a partially
+ // truncated log segment
+ break;
+ }
+ }
+ LOG.info("Deleting log segments older than {} for {} : {}",
+ new Object[] { minTimestampToKeep, getFullyQualifiedName(), purgeList });
+ return deleteLogSegments(purgeList);
+ }
+ });
+ }
+
+ Future<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTxnId(final long minTxIdToKeep) {
+ return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
+ new AbstractFunction1<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
+ @Override
+ public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
+ int numLogSegmentsToProcess;
+
+ if (minTxIdToKeep < 0) {
+ // we are deleting the log, we can remove whole log segments
+ numLogSegmentsToProcess = logSegments.size();
+ } else {
+ numLogSegmentsToProcess = getNumCandidateLogSegmentsToPurge(logSegments);
+ }
+ List<LogSegmentMetadata> purgeList = Lists.newArrayListWithExpectedSize(numLogSegmentsToProcess);
+ for (int iterator = 0; iterator < numLogSegmentsToProcess; iterator++) {
+ LogSegmentMetadata l = logSegments.get(iterator);
+ if ((minTxIdToKeep < 0) ||
+ ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) &&
+ !l.isInProgress() && (l.getLastTxId() < minTxIdToKeep))) {
+ purgeList.add(l);
+ } else {
+ // stop truncating log segments if we find either an inprogress or a partially
+ // truncated log segment
+ break;
+ }
+ }
+ return deleteLogSegments(purgeList);
+ }
+ });
+ }
+
+ private Future<List<LogSegmentMetadata>> setLogSegmentTruncationStatus(
+ final List<LogSegmentMetadata> truncateList,
+ LogSegmentMetadata partialTruncate,
+ DLSN minActiveDLSN) {
+ final List<LogSegmentMetadata> listToTruncate = Lists.newArrayListWithCapacity(truncateList.size() + 1);
+ final List<LogSegmentMetadata> listAfterTruncated = Lists.newArrayListWithCapacity(truncateList.size() + 1);
+ Transaction<Object> updateTxn = metadataUpdater.transaction();
+ for(LogSegmentMetadata l : truncateList) {
+ if (!l.isTruncated()) {
+ LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentTruncated(updateTxn, l);
+ listToTruncate.add(l);
+ listAfterTruncated.add(newSegment);
+ }
+ }
+
+ if (null != partialTruncate && (partialTruncate.isNonTruncated() ||
+ (partialTruncate.isPartiallyTruncated() && (partialTruncate.getMinActiveDLSN().compareTo(minActiveDLSN) < 0)))) {
+ LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentPartiallyTruncated(
+ updateTxn, partialTruncate, minActiveDLSN);
+ listToTruncate.add(partialTruncate);
+ listAfterTruncated.add(newSegment);
+ }
+
+ return updateTxn.execute().map(new AbstractFunction1<Void, List<LogSegmentMetadata>>() {
+ @Override
+ public List<LogSegmentMetadata> apply(Void value) {
+ for (int i = 0; i < listToTruncate.size(); i++) {
+ removeLogSegmentFromCache(listToTruncate.get(i).getSegmentName());
+ LogSegmentMetadata newSegment = listAfterTruncated.get(i);
+ addLogSegmentToCache(newSegment.getSegmentName(), newSegment);
+ }
+ return listAfterTruncated;
+ }
+ });
+ }
+
+ private Future<List<LogSegmentMetadata>> deleteLogSegments(
+ final List<LogSegmentMetadata> logs) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Purging logs for {} : {}", getFullyQualifiedName(), logs);
+ }
+ return FutureUtils.processList(logs,
+ new Function<LogSegmentMetadata, Future<LogSegmentMetadata>>() {
+ @Override
+ public Future<LogSegmentMetadata> apply(LogSegmentMetadata segment) {
+ return deleteLogSegment(segment);
+ }
+ }, scheduler);
+ }
+
+ private Future<LogSegmentMetadata> deleteLogSegment(
+ final LogSegmentMetadata ledgerMetadata) {
+ LOG.info("Deleting ledger {} for {}", ledgerMetadata, getFullyQualifiedName());
+ final Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ promise.addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+ @Override
+ public void onSuccess(LogSegmentMetadata segment) {
+ deleteOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ deleteOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ }
+ });
+ entryStore.deleteLogSegment(ledgerMetadata)
+ .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ FutureUtils.setException(promise, cause);
+ }
+
+ @Override
+ public void onSuccess(LogSegmentMetadata segment) {
+ deleteLogSegmentMetadata(segment, promise);
+ }
+ });
+ return promise;
+ }
+
+ private void deleteLogSegmentMetadata(final LogSegmentMetadata segmentMetadata,
+ final Promise<LogSegmentMetadata> promise) {
+ Transaction<Object> deleteTxn = metadataStore.transaction();
+ metadataStore.deleteLogSegment(deleteTxn, segmentMetadata, new Transaction.OpListener<Void>() {
+ @Override
+ public void onCommit(Void r) {
+ // purge log segment
+ removeLogSegmentFromCache(segmentMetadata.getZNodeName());
+ promise.setValue(segmentMetadata);
+ }
+
+ @Override
+ public void onAbort(Throwable t) {
+ if (t instanceof LogSegmentNotFoundException) {
+ // purge log segment
+ removeLogSegmentFromCache(segmentMetadata.getZNodeName());
+ promise.setValue(segmentMetadata);
+ return;
+ } else {
+ LOG.error("Couldn't purge {} for {}: with error {}",
+ new Object[]{ segmentMetadata, getFullyQualifiedName(), t });
+ promise.setException(t);
+ }
+ }
+ });
+ deleteTxn.execute();
+ }
+
+ @Override
+ public Future<Void> asyncClose() {
+ return Utils.closeSequence(scheduler,
+ lock,
+ logSegmentAllocator);
+ }
+
+ @Override
+ public Future<Void> asyncAbort() {
+ return asyncClose();
+ }
+
+ String completedLedgerZNodeName(long firstTxId, long lastTxId, long logSegmentSeqNo) {
+ if (DistributedLogConstants.LOGSEGMENT_NAME_VERSION == conf.getLogSegmentNameVersion()) {
+ return String.format("%s_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, logSegmentSeqNo);
+ } else {
+ return String.format("%s_%018d_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX,
+ firstTxId, lastTxId);
+ }
+ }
+
+ /**
+ * Get the znode path for a finalize ledger
+ */
+ String completedLedgerZNode(long firstTxId, long lastTxId, long logSegmentSeqNo) {
+ return String.format("%s/%s", logMetadata.getLogSegmentsPath(),
+ completedLedgerZNodeName(firstTxId, lastTxId, logSegmentSeqNo));
+ }
+
+ /**
+ * Get the name of the inprogress znode.
+ *
+ * @return name of the inprogress znode.
+ */
+ String inprogressZNodeName(long logSegmentId, long firstTxId, long logSegmentSeqNo) {
+ if (DistributedLogConstants.LOGSEGMENT_NAME_VERSION == conf.getLogSegmentNameVersion()) {
+ // Lots of the problems are introduced due to different inprogress names with same ledger sequence number.
+ return String.format("%s_%018d", DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX, logSegmentSeqNo);
+ } else {
+ return DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX + "_" + Long.toString(firstTxId, 16);
+ }
+ }
+
+ /**
+ * Get the znode path for the inprogressZNode
+ */
+ String inprogressZNode(long logSegmentId, long firstTxId, long logSegmentSeqNo) {
+ return logMetadata.getLogSegmentsPath() + "/" + inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo);
+ }
+
+ String inprogressZNode(String inprogressZNodeName) {
+ return logMetadata.getLogSegmentsPath() + "/" + inprogressZNodeName;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
new file mode 100644
index 0000000..bf89823
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
@@ -0,0 +1,276 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Ticker;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.IdleReaderException;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.Utils;
+import com.twitter.util.Future;
+import com.twitter.util.Promise;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.versioning.Versioned;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Synchronous Log Reader based on {@link AsyncLogReader}
+ */
+class BKSyncLogReader implements LogReader, AsyncNotification {
+
+ private final BKDistributedLogManager bkdlm;
+ private final BKLogReadHandler readHandler;
+ private final AtomicReference<IOException> readerException =
+ new AtomicReference<IOException>(null);
+ private final int maxReadAheadWaitTime;
+ private Promise<Void> closeFuture;
+ private final Optional<Long> startTransactionId;
+ private boolean positioned = false;
+ private Entry.Reader currentEntry = null;
+
+ // readahead reader
+ ReadAheadEntryReader readAheadReader = null;
+
+ // idle reader settings
+ private final boolean shouldCheckIdleReader;
+ private final int idleErrorThresholdMillis;
+
+ // Stats
+ private final Counter idleReaderError;
+
+ BKSyncLogReader(DistributedLogConfiguration conf,
+ BKDistributedLogManager bkdlm,
+ DLSN startDLSN,
+ Optional<Long> startTransactionId,
+ StatsLogger statsLogger) throws IOException {
+ this.bkdlm = bkdlm;
+ this.readHandler = bkdlm.createReadHandler(
+ Optional.<String>absent(),
+ this,
+ true);
+ this.maxReadAheadWaitTime = conf.getReadAheadWaitTime();
+ this.idleErrorThresholdMillis = conf.getReaderIdleErrorThresholdMillis();
+ this.shouldCheckIdleReader = idleErrorThresholdMillis > 0 && idleErrorThresholdMillis < Integer.MAX_VALUE;
+ this.startTransactionId = startTransactionId;
+
+ // start readahead
+ startReadAhead(startDLSN);
+ if (!startTransactionId.isPresent()) {
+ positioned = true;
+ }
+
+ // Stats
+ StatsLogger syncReaderStatsLogger = statsLogger.scope("sync_reader");
+ idleReaderError = syncReaderStatsLogger.getCounter("idle_reader_error");
+ }
+
+ private void startReadAhead(DLSN startDLSN) throws IOException {
+ readAheadReader = new ReadAheadEntryReader(
+ bkdlm.getStreamName(),
+ startDLSN,
+ bkdlm.getConf(),
+ readHandler,
+ bkdlm.getReaderEntryStore(),
+ bkdlm.getScheduler(),
+ Ticker.systemTicker(),
+ bkdlm.alertStatsLogger);
+ readHandler.registerListener(readAheadReader);
+ readHandler.asyncStartFetchLogSegments()
+ .map(new AbstractFunction1<Versioned<List<LogSegmentMetadata>>, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(Versioned<List<LogSegmentMetadata>> logSegments) {
+ readAheadReader.addStateChangeNotification(BKSyncLogReader.this);
+ readAheadReader.start(logSegments.getValue());
+ return BoxedUnit.UNIT;
+ }
+ });
+ }
+
+ @VisibleForTesting
+ ReadAheadEntryReader getReadAheadReader() {
+ return readAheadReader;
+ }
+
+ @VisibleForTesting
+ BKLogReadHandler getReadHandler() {
+ return readHandler;
+ }
+
+ private Entry.Reader readNextEntry(boolean nonBlocking) throws IOException {
+ Entry.Reader entry = null;
+ if (nonBlocking) {
+ return readAheadReader.getNextReadAheadEntry(0L, TimeUnit.MILLISECONDS);
+ } else {
+ while (!readAheadReader.isReadAheadCaughtUp()
+ && null == readerException.get()
+ && null == entry) {
+ entry = readAheadReader.getNextReadAheadEntry(maxReadAheadWaitTime, TimeUnit.MILLISECONDS);
+ }
+ if (null != entry) {
+ return entry;
+ }
+ // reader is caught up
+ if (readAheadReader.isReadAheadCaughtUp()
+ && null == readerException.get()) {
+ entry = readAheadReader.getNextReadAheadEntry(maxReadAheadWaitTime, TimeUnit.MILLISECONDS);
+ }
+ return entry;
+ }
+ }
+
+ private void markReaderAsIdle() throws IdleReaderException {
+ idleReaderError.inc();
+ IdleReaderException ire = new IdleReaderException("Sync reader on stream "
+ + readHandler.getFullyQualifiedName()
+ + " is idle for more than " + idleErrorThresholdMillis + " ms");
+ readerException.compareAndSet(null, ire);
+ throw ire;
+ }
+
+ @Override
+ public synchronized LogRecordWithDLSN readNext(boolean nonBlocking)
+ throws IOException {
+ if (null != readerException.get()) {
+ throw readerException.get();
+ }
+ LogRecordWithDLSN record = doReadNext(nonBlocking);
+ // no record is returned, check if the reader becomes idle
+ if (null == record && shouldCheckIdleReader) {
+ if (readAheadReader.getNumCachedEntries() <= 0 &&
+ readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS)) {
+ markReaderAsIdle();
+ }
+ }
+ return record;
+ }
+
+ private LogRecordWithDLSN doReadNext(boolean nonBlocking) throws IOException {
+ LogRecordWithDLSN record = null;
+
+ do {
+ // fetch one record until we don't find any entry available in the readahead cache
+ while (null == record) {
+ if (null == currentEntry) {
+ currentEntry = readNextEntry(nonBlocking);
+ if (null == currentEntry) {
+ return null;
+ }
+ }
+ record = currentEntry.nextRecord();
+ if (null == record) {
+ currentEntry = null;
+ }
+ }
+
+ // check if we reached the end of stream
+ if (record.isEndOfStream()) {
+ EndOfStreamException eos = new EndOfStreamException("End of Stream Reached for "
+ + readHandler.getFullyQualifiedName());
+ readerException.compareAndSet(null, eos);
+ throw eos;
+ }
+ // skip control records
+ if (record.isControl()) {
+ record = null;
+ continue;
+ }
+ if (!positioned) {
+ if (record.getTransactionId() < startTransactionId.get()) {
+ record = null;
+ continue;
+ } else {
+ positioned = true;
+ break;
+ }
+ } else {
+ break;
+ }
+ } while (true);
+ return record;
+ }
+
+ @Override
+ public synchronized List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords)
+ throws IOException {
+ LinkedList<LogRecordWithDLSN> retList =
+ new LinkedList<LogRecordWithDLSN>();
+
+ int numRead = 0;
+ LogRecordWithDLSN record = readNext(nonBlocking);
+ while ((null != record)) {
+ retList.add(record);
+ numRead++;
+ if (numRead >= numLogRecords) {
+ break;
+ }
+ record = readNext(nonBlocking);
+ }
+ return retList;
+ }
+
+ @Override
+ public Future<Void> asyncClose() {
+ Promise<Void> closePromise;
+ synchronized (this) {
+ if (null != closeFuture) {
+ return closeFuture;
+ }
+ closeFuture = closePromise = new Promise<Void>();
+ }
+ readHandler.unregisterListener(readAheadReader);
+ readAheadReader.removeStateChangeNotification(this);
+ Utils.closeSequence(bkdlm.getScheduler(), true,
+ readAheadReader,
+ readHandler
+ ).proxyTo(closePromise);
+ return closePromise;
+ }
+
+ @Override
+ public void close() throws IOException {
+ FutureUtils.result(asyncClose());
+ }
+
+ //
+ // Notification From ReadHandler
+ //
+
+ @Override
+ public void notifyOnError(Throwable cause) {
+ if (cause instanceof IOException) {
+ readerException.compareAndSet(null, (IOException) cause);
+ } else {
+ readerException.compareAndSet(null, new IOException(cause));
+ }
+ }
+
+ @Override
+ public void notifyOnOperationComplete() {
+ // no-op
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java
new file mode 100644
index 0000000..7d33d12
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java
@@ -0,0 +1,113 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.util.FutureUtils;
+
+import java.io.IOException;
+import java.util.List;
+
+class BKSyncLogWriter extends BKAbstractLogWriter implements LogWriter {
+
+ public BKSyncLogWriter(DistributedLogConfiguration conf,
+ DynamicDistributedLogConfiguration dynConf,
+ BKDistributedLogManager bkdlm) {
+ super(conf, dynConf, bkdlm);
+ }
+ /**
+ * Write log records to the stream.
+ *
+ * @param record operation
+ */
+ @Override
+ public void write(LogRecord record) throws IOException {
+ getLedgerWriter(record.getTransactionId(), false).write(record);
+ }
+
+ /**
+ * Write edits logs operation to the stream.
+ *
+ * @param records list of records
+ */
+ @Override
+ @Deprecated
+ public int writeBulk(List<LogRecord> records) throws IOException {
+ return getLedgerWriter(records.get(0).getTransactionId(), false).writeBulk(records);
+ }
+
+ /**
+ * Flushes all the data up to this point,
+ * adds the end of stream marker and marks the stream
+ * as read-only in the metadata. No appends to the
+ * stream will be allowed after this point
+ */
+ @Override
+ public void markEndOfStream() throws IOException {
+ FutureUtils.result(getLedgerWriter(DistributedLogConstants.MAX_TXID, true).markEndOfStream());
+ closeAndComplete();
+ }
+
+ /**
+ * All data that has been written to the stream so far will be flushed.
+ * New data can be still written to the stream while flush is ongoing.
+ */
+ @Override
+ public long setReadyToFlush() throws IOException {
+ checkClosedOrInError("setReadyToFlush");
+ long highestTransactionId = 0;
+ BKLogSegmentWriter writer = getCachedLogWriter();
+ if (null != writer) {
+ highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.flush()));
+ }
+ return highestTransactionId;
+ }
+
+ /**
+ * Commit data that is already flushed.
+ * <p/>
+ * This API is optional as the writer implements a policy for automatically syncing
+ * the log records in the buffer. The buffered edits can be flushed when the buffer
+ * becomes full or a certain period of time is elapsed.
+ */
+ @Override
+ public long flushAndSync() throws IOException {
+ checkClosedOrInError("flushAndSync");
+
+ LOG.debug("FlushAndSync Started");
+ long highestTransactionId = 0;
+ BKLogSegmentWriter writer = getCachedLogWriter();
+ if (null != writer) {
+ highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.commit()));
+ LOG.debug("FlushAndSync Completed");
+ } else {
+ LOG.debug("FlushAndSync Completed - Nothing to Flush");
+ }
+ return highestTransactionId;
+ }
+
+ /**
+ * Close the stream without necessarily flushing immediately.
+ * This may be called if the stream is in error such as after a
+ * previous write or close threw an exception.
+ */
+ @Override
+ public void abort() throws IOException {
+ super.abort();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
new file mode 100644
index 0000000..6ed662b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
@@ -0,0 +1,90 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+
+import java.util.concurrent.TimeUnit;
+
+class BKTransmitPacket {
+
+ private final EntryBuffer recordSet;
+ private final long transmitTime;
+ private final Promise<Integer> transmitComplete;
+
+ BKTransmitPacket(EntryBuffer recordSet) {
+ this.recordSet = recordSet;
+ this.transmitTime = System.nanoTime();
+ this.transmitComplete = new Promise<Integer>();
+ }
+
+ EntryBuffer getRecordSet() {
+ return recordSet;
+ }
+
+ Promise<Integer> getTransmitFuture() {
+ return transmitComplete;
+ }
+
+ /**
+ * Complete the transmit with result code <code>transmitRc</code>.
+ * <p>It would notify all the waiters that are waiting via {@link #awaitTransmitComplete(long, TimeUnit)}
+ * or {@link #addTransmitCompleteListener(FutureEventListener)}.
+ *
+ * @param transmitResult
+ * transmit result code.
+ */
+ public void notifyTransmitComplete(int transmitResult) {
+ transmitComplete.setValue(transmitResult);
+ }
+
+ /**
+ * Register a transmit complete listener.
+ * <p>The listener will be triggered with transmit result when transmit completes.
+ * The method should be non-blocking.
+ *
+ * @param transmitCompleteListener
+ * listener on transmit completion
+ * @see #awaitTransmitComplete(long, TimeUnit)
+ */
+ void addTransmitCompleteListener(FutureEventListener<Integer> transmitCompleteListener) {
+ transmitComplete.addEventListener(transmitCompleteListener);
+ }
+
+ /**
+ * Await for the transmit to be complete
+ *
+ * @param timeout
+ * wait timeout
+ * @param unit
+ * wait timeout unit
+ */
+ int awaitTransmitComplete(long timeout, TimeUnit unit)
+ throws Exception {
+ return Await.result(transmitComplete,
+ Duration.fromTimeUnit(timeout, unit));
+ }
+
+ public long getTransmitTime() {
+ return transmitTime;
+ }
+
+}
[06/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
new file mode 100644
index 0000000..0cb608f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
@@ -0,0 +1,3528 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.bk.QuorumConfig;
+import org.apache.distributedlog.feature.DefaultFeatureProvider;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.net.DNSResolverForRacks;
+import org.apache.distributedlog.net.DNSResolverForRows;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.net.DNSToSwitchMapping;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.configuration.SystemConfiguration;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Set;
+
+/**
+ * DistributedLog Configuration.
+ * <p>
+ * DistributedLog configuration is basically a properties based configuration, which extends from
+ * Apache commons {@link CompositeConfiguration}. All the DL settings are in camel case and prefixed
+ * with a meaningful component name. for example, `zkSessionTimeoutSeconds` means <i>SessionTimeoutSeconds</i>
+ * for component `zk`.
+ *
+ * <h3>BookKeeper Configuration</h3>
+ *
+ * BookKeeper client configuration settings could be loaded via DistributedLog configuration. All those
+ * settings are prefixed with <i>`bkc.`</i>. For example, <i>bkc.zkTimeout</i> in distributedlog configuration
+ * will be applied as <i>`zkTimeout`</i> in bookkeeper client configuration.
+ *
+ * <h3>How to load configuration</h3>
+ *
+ * The default distributedlog configuration is constructed by instantiated a new instance. This
+ * distributedlog configuration will automatically load the settings that specified via
+ * {@link SystemConfiguration}.
+ *
+ * <pre>
+ * DistributedLogConfiguration conf = new DistributedLogConfiguration();
+ * </pre>
+ *
+ * The recommended way is to load configuration from URL that points to a configuration file
+ * ({@link #loadConf(URL)}).
+ *
+ * <pre>
+ * String configFile = "/path/to/distributedlog/conf/file";
+ * DistributedLogConfiguration conf = new DistributedLogConfiguration();
+ * conf.loadConf(new File(configFile).toURI().toURL());
+ * </pre>
+ *
+ * @see org.apache.bookkeeper.conf.ClientConfiguration
+ */
+public class DistributedLogConfiguration extends CompositeConfiguration {
+ static final Logger LOG = LoggerFactory.getLogger(DistributedLogConfiguration.class);
+
+ private static ClassLoader defaultLoader;
+
+ static {
+ defaultLoader = Thread.currentThread().getContextClassLoader();
+ if (null == defaultLoader) {
+ defaultLoader = DistributedLogConfiguration.class.getClassLoader();
+ }
+ }
+
+ //
+ // ZooKeeper Related Settings
+ //
+
+ public static final String BKDL_ZK_ACL_ID = "zkAclId";
+ public static final String BKDL_ZK_ACL_ID_DEFAULT = null;
+ public static final String BKDL_ZK_SESSION_TIMEOUT_SECONDS = "zkSessionTimeoutSeconds";
+ public static final int BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT = 30;
+ public static final String BKDL_ZK_REQUEST_RATE_LIMIT = "zkRequestRateLimit";
+ public static final double BKDL_ZK_REQUEST_RATE_LIMIT_DEFAULT = 0;
+ public static final String BKDL_ZK_NUM_RETRIES = "zkNumRetries";
+ public static final int BKDL_ZK_NUM_RETRIES_DEFAULT = 3;
+ public static final String BKDL_ZK_RETRY_BACKOFF_START_MILLIS = "zkRetryStartBackoffMillis";
+ public static final int BKDL_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT = 5000;
+ public static final String BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS = "zkRetryMaxBackoffMillis";
+ public static final int BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT = 30000;
+ public static final String BKDL_ZKCLIENT_NUM_RETRY_THREADS = "zkcNumRetryThreads";
+ public static final int BKDL_ZKCLIENT_NUM_RETRY_THREADS_DEFAULT = 1;
+
+ //
+ // BookKeeper Related Settings
+ //
+
+ // BookKeeper zookeeper settings
+ public static final String BKDL_BKCLIENT_ZK_SESSION_TIMEOUT = "bkcZKSessionTimeoutSeconds";
+ public static final int BKDL_BKCLIENT_ZK_SESSION_TIMEOUT_DEFAULT = 30;
+ public static final String BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT = "bkcZKRequestRateLimit";
+ public static final double BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT_DEFAULT = 0;
+ public static final String BKDL_BKCLIENT_ZK_NUM_RETRIES = "bkcZKNumRetries";
+ public static final int BKDL_BKCLIENT_ZK_NUM_RETRIES_DEFAULT = 3;
+ public static final String BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS = "bkcZKRetryStartBackoffMillis";
+ public static final int BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT = 5000;
+ public static final String BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS = "bkcZKRetryMaxBackoffMillis";
+ public static final int BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT = 30000;
+
+ // Bookkeeper ensemble placement settings
+ // Bookkeeper ensemble size
+ public static final String BKDL_BOOKKEEPER_ENSEMBLE_SIZE = "bkcEnsembleSize";
+ // @Deprecated
+ public static final String BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD = "ensemble-size";
+ public static final int BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT = 3;
+ // Bookkeeper write quorum size
+ public static final String BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE = "bkcWriteQuorumSize";
+ // @Deprecated
+ public static final String BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD = "write-quorum-size";
+ public static final int BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT = 3;
+ // Bookkeeper ack quorum size
+ public static final String BKDL_BOOKKEEPER_ACK_QUORUM_SIZE = "bkcAckQuorumSize";
+ // @Deprecated
+ public static final String BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD = "ack-quorum-size";
+ public static final int BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT = 2;
+ public static final String BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT = "bkRowAwareEnsemblePlacement";
+ public static final String BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_OLD = "row-aware-ensemble-placement";
+ public static final boolean BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_DEFAULT = false;
+ public static final String BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS = "bkEnsemblePlacementDnsResolverClass";
+ public static final String BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS_DEFAULT =
+ DNSResolverForRacks.class.getName();
+ public static final String BKDL_BK_DNS_RESOLVER_OVERRIDES = "dnsResolverOverrides";
+ public static final String BKDL_BK_DNS_RESOLVER_OVERRIDES_DEFAULT = "";
+
+ // General Settings
+ // @Deprecated
+ public static final String BKDL_BOOKKEEPER_DIGEST_PW = "digestPw";
+ public static final String BKDL_BOOKKEEPER_DIGEST_PW_DEFAULT = "";
+ public static final String BKDL_BKCLIENT_NUM_IO_THREADS = "bkcNumIOThreads";
+ public static final String BKDL_TIMEOUT_TIMER_TICK_DURATION_MS = "timerTickDuration";
+ public static final long BKDL_TIMEOUT_TIMER_TICK_DURATION_MS_DEFAULT = 100;
+ public static final String BKDL_TIMEOUT_TIMER_NUM_TICKS = "timerNumTicks";
+ public static final int BKDL_TIMEOUT_TIMER_NUM_TICKS_DEFAULT = 1024;
+
+ //
+ // Deprecated BookKeeper Settings (in favor of "bkc." style bookkeeper settings)
+ //
+
+ public static final String BKDL_BKCLIENT_READ_TIMEOUT = "bkcReadTimeoutSeconds";
+ public static final int BKDL_BKCLIENT_READ_TIMEOUT_DEFAULT = 10;
+ public static final String BKDL_BKCLIENT_WRITE_TIMEOUT = "bkcWriteTimeoutSeconds";
+ public static final int BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT = 10;
+ public static final String BKDL_BKCLIENT_NUM_WORKER_THREADS = "bkcNumWorkerThreads";
+ public static final int BKDL_BKCLEINT_NUM_WORKER_THREADS_DEFAULT = 1;
+
+ //
+ // DL General Settings
+ //
+
+ // Executor Parameters
+ public static final String BKDL_NUM_WORKER_THREADS = "numWorkerThreads";
+ public static final String BKDL_NUM_READAHEAD_WORKER_THREADS = "numReadAheadWorkerThreads";
+ public static final String BKDL_NUM_LOCKSTATE_THREADS = "numLockStateThreads";
+ public static final String BKDL_NUM_RESOURCE_RELEASE_THREADS = "numResourceReleaseThreads";
+ public static final String BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS = "schedulerShutdownTimeoutMs";
+ public static final int BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS_DEFAULT = 5000;
+ public static final String BKDL_USE_DAEMON_THREAD = "useDaemonThread";
+ public static final boolean BKDL_USE_DAEMON_THREAD_DEFAULT = false;
+
+ // Metadata Parameters
+ public static final String BKDL_LEDGER_METADATA_LAYOUT_VERSION = "ledgerMetadataLayoutVersion";
+ public static final String BKDL_LEDGER_METADATA_LAYOUT_VERSION_OLD = "ledger-metadata-layout";
+ public static final int BKDL_LEDGER_METADATA_LAYOUT_VERSION_DEFAULT =
+ LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
+ public static final String BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK = "ledgerMetadataSkipMinVersionCheck";
+ public static final boolean BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK_DEFAULT = false;
+ public static final String BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER = "firstLogsegmentSequenceNumber";
+ public static final String BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_OLD = "first-logsegment-sequence-number";
+ public static final long BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_DEFAULT =
+ DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO;
+ public static final String BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED = "logSegmentSequenceNumberValidationEnabled";
+ public static final boolean BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED_DEFAULT = true;
+ public static final String BKDL_ENABLE_RECORD_COUNTS = "enableRecordCounts";
+ public static final boolean BKDL_ENABLE_RECORD_COUNTS_DEFAULT = true;
+ public static final String BKDL_MAXID_SANITYCHECK = "maxIdSanityCheck";
+ public static final boolean BKDL_MAXID_SANITYCHECK_DEFAULT = true;
+ public static final String BKDL_ENCODE_REGION_ID_IN_VERSION = "encodeRegionIDInVersion";
+ public static final boolean BKDL_ENCODE_REGION_ID_IN_VERSION_DEFAULT = false;
+ // (@Deprecated)
+ public static final String BKDL_LOGSEGMENT_NAME_VERSION = "logSegmentNameVersion";
+ public static final int BKDL_LOGSEGMENT_NAME_VERSION_DEFAULT = DistributedLogConstants.LOGSEGMENT_NAME_VERSION;
+ // (@Derepcated) Name for the default (non-partitioned) stream
+ public static final String BKDL_UNPARTITIONED_STREAM_NAME = "unpartitionedStreamName";
+ public static final String BKDL_UNPARTITIONED_STREAM_NAME_DEFAULT = "<default>";
+
+ // Log Segment Cache Parameters
+ public static final String BKDL_LOGSEGMENT_CACHE_TTL_MS = "logSegmentCacheTTLMs";
+ public static final long BKDL_LOGSEGMENT_CACHE_TTL_MS_DEFAULT = 600000; // 10 mins
+ public static final String BKDL_LOGSEGMENT_CACHE_MAX_SIZE = "logSegmentCacheMaxSize";
+ public static final long BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT = 10000;
+ public static final String BKDL_LOGSEGMENT_CACHE_ENABLED = "logSegmentCacheEnabled";
+ public static final boolean BKDL_LOGSEGMENT_CACHE_ENABLED_DEFAULT = true;
+
+ //
+ // DL Writer Settings
+ //
+
+ // General Settings
+ public static final String BKDL_CREATE_STREAM_IF_NOT_EXISTS = "createStreamIfNotExists";
+ public static final boolean BKDL_CREATE_STREAM_IF_NOT_EXISTS_DEFAULT = true;
+ public static final String BKDL_LOG_FLUSH_TIMEOUT = "logFlushTimeoutSeconds";
+ public static final int BKDL_LOG_FLUSH_TIMEOUT_DEFAULT = 30;
+ /**
+ * CompressionCodec.Type String to use (See CompressionUtils)
+ * --------------------- ------------------------------------
+ * NONE none
+ * LZ4 lz4
+ * UNKNOWN any other instance of String.class
+ */
+ public static final String BKDL_COMPRESSION_TYPE = "compressionType";
+ public static final String BKDL_COMPRESSION_TYPE_DEFAULT = "none";
+ public static final String BKDL_FAILFAST_ON_STREAM_NOT_READY = "failFastOnStreamNotReady";
+ public static final boolean BKDL_FAILFAST_ON_STREAM_NOT_READY_DEFAULT = false;
+ public static final String BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR = "disableRollingOnLogSegmentError";
+ public static final boolean BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR_DEFAULT = false;
+
+ // Durability Settings
+ public static final String BKDL_IS_DURABLE_WRITE_ENABLED = "isDurableWriteEnabled";
+ public static final boolean BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT = true;
+
+ // Transmit Settings
+ public static final String BKDL_OUTPUT_BUFFER_SIZE = "writerOutputBufferSize";
+ public static final String BKDL_OUTPUT_BUFFER_SIZE_OLD = "output-buffer-size";
+ public static final int BKDL_OUTPUT_BUFFER_SIZE_DEFAULT = 1024;
+ public static final String BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS = "periodicFlushFrequencyMilliSeconds";
+ public static final int BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT = 0;
+ public static final String BKDL_ENABLE_IMMEDIATE_FLUSH = "enableImmediateFlush";
+ public static final boolean BKDL_ENABLE_IMMEDIATE_FLUSH_DEFAULT = false;
+ public static final String BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS = "minimumDelayBetweenImmediateFlushMilliSeconds";
+ public static final int BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT = 0;
+ public static final String BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS = "periodicKeepAliveMilliSeconds";
+ public static final int BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS_DEFAULT = 0;
+
+ // Retention/Truncation Settings
+ public static final String BKDL_RETENTION_PERIOD_IN_HOURS = "logSegmentRetentionHours";
+ public static final String BKDL_RETENTION_PERIOD_IN_HOURS_OLD = "retention-size";
+ public static final int BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT = 72;
+ public static final String BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION = "explicitTruncationByApp";
+ public static final boolean BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION_DEFAULT = false;
+
+ // Log Segment Rolling Settings
+ public static final String BKDL_ROLLING_INTERVAL_IN_MINUTES = "logSegmentRollingMinutes";
+ public static final String BKDL_ROLLING_INTERVAL_IN_MINUTES_OLD = "rolling-interval";
+ public static final int BKDL_ROLLING_INTERVAL_IN_MINUTES_DEFAULT = 120;
+ public static final String BKDL_MAX_LOGSEGMENT_BYTES = "maxLogSegmentBytes";
+ public static final int BKDL_MAX_LOGSEGMENT_BYTES_DEFAULT = 256 * 1024 * 1024; // default 256MB
+ public static final String BKDL_LOGSEGMENT_ROLLING_CONCURRENCY = "logSegmentRollingConcurrency";
+ public static final int BKDL_LOGSEGMENT_ROLLING_CONCURRENCY_DEFAULT = 1;
+
+ // Lock Settings
+ public static final String BKDL_WRITE_LOCK_ENABLED = "writeLockEnabled";
+ public static final boolean BKDL_WRITE_LOCK_ENABLED_DEFAULT = true;
+ public static final String BKDL_LOCK_TIMEOUT = "lockTimeoutSeconds";
+ public static final long BKDL_LOCK_TIMEOUT_DEFAULT = 30;
+ public static final String BKDL_LOCK_REACQUIRE_TIMEOUT = "lockReacquireTimeoutSeconds";
+ public static final long BKDL_LOCK_REACQUIRE_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_REACQUIRE_TIMEOUT_DEFAULT;
+ public static final String BKDL_LOCK_OP_TIMEOUT = "lockOpTimeoutSeconds";
+ public static final long BKDL_LOCK_OP_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_OP_TIMEOUT_DEFAULT;
+
+ // Ledger Allocator Settings
+ public static final String BKDL_ENABLE_LEDGER_ALLOCATOR_POOL = "enableLedgerAllocatorPool";
+ public static final boolean BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT = false;
+ public static final String BKDL_LEDGER_ALLOCATOR_POOL_PATH = "ledgerAllocatorPoolPath";
+ public static final String BKDL_LEDGER_ALLOCATOR_POOL_PATH_DEFAULT = DistributedLogConstants.ALLOCATION_POOL_NODE;
+ public static final String BKDL_LEDGER_ALLOCATOR_POOL_NAME = "ledgerAllocatorPoolName";
+ public static final String BKDL_LEDGER_ALLOCATOR_POOL_NAME_DEFAULT = null;
+ public static final String BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE = "ledgerAllocatorPoolCoreSize";
+ public static final int BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE_DEFAULT = 20;
+
+ // Write Limit Settings
+ public static final String BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT = "perWriterOutstandingWriteLimit";
+ public static final int BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT = "globalOutstandingWriteLimit";
+ public static final int BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE = "outstandingWriteLimitDarkmode";
+ public static final boolean BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE_DEFAULT = true;
+
+ //
+ // DL Reader Settings
+ //
+
+ // General Settings
+ public static final String BKDL_READLAC_OPTION = "readLACLongPoll";
+ public static final int BKDL_READLAC_OPTION_DEFAULT = 3; //BKLogPartitionReadHandler.ReadLACOption.READENTRYPIGGYBACK_SEQUENTIAL.value
+ public static final String BKDL_READLACLONGPOLL_TIMEOUT = "readLACLongPollTimeout";
+ public static final int BKDL_READLACLONGPOLL_TIMEOUT_DEFAULT = 1000;
+ public static final String BKDL_DESERIALIZE_RECORDSET_ON_READS = "deserializeRecordSetOnReads";
+ public static final boolean BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT = true;
+
+ // Idle reader settings
+ public static final String BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS = "readerIdleWarnThresholdMillis";
+ public static final int BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS_DEFAULT = 120000;
+ public static final String BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS = "readerIdleErrorThresholdMillis";
+ public static final int BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT = Integer.MAX_VALUE;
+
+ // Reader constraint settings
+ public static final String BKDL_READER_IGNORE_TRUNCATION_STATUS = "ignoreTruncationStatus";
+ public static final boolean BKDL_READER_IGNORE_TRUNCATION_STATUS_DEFAULT = false;
+ public static final String BKDL_READER_ALERT_POSITION_ON_TRUNCATED = "alertPositionOnTruncated";
+ public static final boolean BKDL_READER_ALERT_POSITION_ON_TRUNCATED_DEFAULT = true;
+ public static final String BKDL_READER_POSITION_GAP_DETECTION_ENABLED = "positionGapDetectionEnabled";
+ public static final boolean BKDL_READER_POSITION_GAP_DETECTION_ENABLED_DEFAULT = false;
+
+ // Read ahead related parameters
+ public static final String BKDL_ENABLE_READAHEAD = "enableReadAhead";
+ public static final boolean BKDL_ENABLE_READAHEAD_DEFAULT = true;
+ public static final String BKDL_ENABLE_FORCEREAD = "enableForceRead";
+ public static final boolean BKDL_ENABLE_FORCEREAD_DEFAULT = true;
+ public static final String BKDL_READAHEAD_MAX_RECORDS = "readAheadMaxRecords";
+ public static final String BKDL_READAHEAD_MAX_RECORDS_OLD = "ReadAheadMaxEntries";
+ public static final int BKDL_READAHEAD_MAX_RECORDS_DEFAULT = 10;
+ public static final String BKDL_READAHEAD_BATCHSIZE = "readAheadBatchSize";
+ public static final String BKDL_READAHEAD_BATCHSIZE_OLD = "ReadAheadBatchSize";
+ public static final int BKDL_READAHEAD_BATCHSIZE_DEFAULT = 2;
+ public static final String BKDL_READAHEAD_WAITTIME = "readAheadWaitTime";
+ public static final String BKDL_READAHEAD_WAITTIME_OLD = "ReadAheadWaitTime";
+ public static final int BKDL_READAHEAD_WAITTIME_DEFAULT = 200;
+ public static final String BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM = "readAheadWaitTimeOnEndOfStream";
+ public static final String BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_OLD = "ReadAheadWaitTimeOnEndOfStream";
+ public static final int BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_DEFAULT = 10000;
+ public static final String BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS =
+ "readAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis";
+ public static final int BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS_DEFAULT = 10000;
+ public static final String BKDL_READAHEAD_SKIP_BROKEN_ENTRIES = "readAheadSkipBrokenEntries";
+ public static final boolean BKDL_READAHEAD_SKIP_BROKEN_ENTRIES_DEFAULT = false;
+ public static final String BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT = "numPrefetchEntriesPerLogSegment";
+ public static final int BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT = 4;
+ public static final String BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT = "maxPrefetchEntriesPerLogSegment";
+ public static final int BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT = 32;
+
+ // Scan Settings
+ public static final String BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "firstNumEntriesEachPerLastRecordScan";
+ public static final int BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 2;
+ public static final String BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "maxNumEntriesPerReadLastRecordScan";
+ public static final int BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 16;
+
+ // Log Existence Settings
+ public static final String BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS = "checkLogExistenceBackoffStartMillis";
+ public static final int BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS_DEFAULT = 200;
+ public static final String BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS = "checkLogExistenceBackoffMaxMillis";
+ public static final int BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS_DEFAULT = 1000;
+
+ //
+ // Tracing/Stats Settings
+ //
+
+ public static final String BKDL_TRACE_READAHEAD_DELIVERY_LATENCY = "traceReadAheadDeliveryLatency";
+ public static final boolean BKDL_TRACE_READAHEAD_DELIVERY_LATENCY_DEFAULT = false;
+ public static final String BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS = "metadataLatencyWarnThresholdMs";
+ public static final long BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
+ public static final String BKDL_DATA_LATENCY_WARN_THRESHOLD_MS = "dataLatencyWarnThresholdMs";
+ public static final long BKDL_DATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = 2 * DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
+ public static final String BKDL_TRACE_READAHEAD_METADATA_CHANGES = "traceReadAheadMetadataChanges";
+ public static final boolean BKDL_TRACE_READAHEAD_MEATDATA_CHANGES_DEFAULT = false;
+ public final static String BKDL_ENABLE_TASK_EXECUTION_STATS = "enableTaskExecutionStats";
+ public final static boolean BKDL_ENABLE_TASK_EXECUTION_STATS_DEFAULT = false;
+ public final static String BKDL_TASK_EXECUTION_WARN_TIME_MICROS = "taskExecutionWarnTimeMicros";
+ public final static long BKDL_TASK_EXECUTION_WARN_TIME_MICROS_DEFAULT = 100000;
+ public static final String BKDL_ENABLE_PERSTREAM_STAT = "enablePerStreamStat";
+ public static final boolean BKDL_ENABLE_PERSTREAM_STAT_DEFAULT = false;
+
+ //
+ // Settings for Feature Providers
+ //
+
+ public static final String BKDL_FEATURE_PROVIDER_CLASS = "featureProviderClass";
+
+ //
+ // Settings for Configuration Based Feature Provider
+ //
+
+ public static final String BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH = "fileFeatureProviderBaseConfigPath";
+ public static final String BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH_DEFAULT = "decider.conf";
+ public static final String BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH = "fileFeatureProviderOverlayConfigPath";
+ public static final String BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH_DEFAULT = null;
+
+ //
+ // Settings for Namespaces
+ //
+
+ public static final String BKDL_FEDERATED_NAMESPACE_ENABLED = "federatedNamespaceEnabled";
+ public static final boolean BKDL_FEDERATED_NAMESPACE_ENABLED_DEFAULT = false;
+ public static final String BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE = "federatedMaxLogsPerSubnamespace";
+ public static final int BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE_DEFAULT = 15000;
+ public static final String BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS = "federatedCheckExistenceWhenCacheMiss";
+ public static final boolean BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS_DEFAULT = true;
+
+ // Settings for Configurations
+
+ public static final String BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC = "dynamicConfigReloadIntervalSec";
+ public static final int BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC_DEFAULT = 60;
+ public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS = "streamConfigRouterClass";
+ public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS_DEFAULT = "org.apache.distributedlog.service.config.IdentityConfigRouter";
+
+ // Settings for RateLimit (used by distributedlog-service)
+
+ public static final String BKDL_BPS_SOFT_WRITE_LIMIT = "bpsSoftWriteLimit";
+ public static final int BKDL_BPS_SOFT_WRITE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_BPS_HARD_WRITE_LIMIT = "bpsHardWriteLimit";
+ public static final int BKDL_BPS_HARD_WRITE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_RPS_SOFT_WRITE_LIMIT = "rpsSoftWriteLimit";
+ public static final int BKDL_RPS_SOFT_WRITE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_RPS_HARD_WRITE_LIMIT = "rpsHardWriteLimit";
+ public static final int BKDL_RPS_HARD_WRITE_LIMIT_DEFAULT = -1;
+
+ // Rate and resource limits: per shard
+
+ public static final String BKDL_RPS_SOFT_SERVICE_LIMIT = "rpsSoftServiceLimit";
+ public static final int BKDL_RPS_SOFT_SERVICE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_RPS_HARD_SERVICE_LIMIT = "rpsHardServiceLimit";
+ public static final int BKDL_RPS_HARD_SERVICE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT = "rpsStreamAcquireServiceLimit";
+ public static final int BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_BPS_SOFT_SERVICE_LIMIT = "bpsSoftServiceLimit";
+ public static final int BKDL_BPS_SOFT_SERVICE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_BPS_HARD_SERVICE_LIMIT = "bpsHardServiceLimit";
+ public static final int BKDL_BPS_HARD_SERVICE_LIMIT_DEFAULT = -1;
+ public static final String BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT = "bpsStreamAcquireServiceLimit";
+ public static final int BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT = -1;
+
+ // Settings for Partitioning
+
+ public static final String BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY = "maxAcquiredPartitionsPerProxy";
+ public static final int BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT = -1;
+
+ public static final String BKDL_MAX_CACHED_PARTITIONS_PER_PROXY = "maxCachedPartitionsPerProxy";
+ public static final int BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT = -1;
+
+ //
+ // Settings for Error Injection
+ //
+ public static final String BKDL_EI_INJECT_WRITE_DELAY = "eiInjectWriteDelay";
+ public static final boolean BKDL_EI_INJECT_WRITE_DELAY_DEFAULT = false;
+ public static final String BKDL_EI_INJECTED_WRITE_DELAY_PERCENT = "eiInjectedWriteDelayPercent";
+ public static final double BKDL_EI_INJECTED_WRITE_DELAY_PERCENT_DEFAULT = 0.0;
+ public static final String BKDL_EI_INJECTED_WRITE_DELAY_MS = "eiInjectedWriteDelayMs";
+ public static final int BKDL_EI_INJECTED_WRITE_DELAY_MS_DEFAULT = 0;
+ public static final String BKDL_EI_INJECT_READAHEAD_STALL = "eiInjectReadAheadStall";
+ public static final boolean BKDL_EI_INJECT_READAHEAD_STALL_DEFAULT = false;
+ public static final String BKDL_EI_INJECT_READAHEAD_DELAY = "eiInjectReadAheadDelay";
+ public static final boolean BKDL_EI_INJECT_READAHEAD_DELAY_DEFAULT = false;
+ public static final String BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS = "eiInjectMaxReadAheadDelayMs";
+ public static final int BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS_DEFAULT = 0;
+ public static final String BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT = "eiInjectReadAheadDelayPercent";
+ public static final int BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT_DEFAULT = 10;
+ public static final String BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES = "eiInjectReadAheadBrokenEntries";
+ public static final boolean BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES_DEFAULT = false;
+
+ // Whitelisted stream-level configuration settings.
+ private static final Set<String> streamSettings = Sets.newHashSet(
+ BKDL_READER_POSITION_GAP_DETECTION_ENABLED,
+ BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS,
+ BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS,
+ BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
+ BKDL_ENABLE_IMMEDIATE_FLUSH
+ );
+
+ /**
+ * Construct distributedlog configuration with default settings.
+ * It also loads the settings from system properties.
+ */
+ public DistributedLogConfiguration() {
+ super();
+ // add configuration for system properties
+ addConfiguration(new SystemConfiguration());
+ }
+
+ /**
+ * You can load configurations in precedence order. The first one takes
+ * precedence over any loaded later.
+ *
+ * @param confURL Configuration URL
+ */
+ public void loadConf(URL confURL) throws ConfigurationException {
+ Configuration loadedConf = new PropertiesConfiguration(confURL);
+ addConfiguration(loadedConf);
+ }
+
+ /**
+ * You can load configuration from other configuration
+ *
+ * @param baseConf Other Configuration
+ */
+ public void loadConf(DistributedLogConfiguration baseConf) {
+ addConfiguration(baseConf);
+ }
+
+ /**
+ * Load configuration from other configuration object
+ *
+ * @param otherConf Other configuration object
+ */
+ public void loadConf(Configuration otherConf) {
+ addConfiguration(otherConf);
+ }
+
+ /**
+ * Load whitelisted stream configuration from another configuration object
+ *
+ * @param streamConfiguration stream configuration overrides
+ */
+ public void loadStreamConf(Optional<DistributedLogConfiguration> streamConfiguration) {
+ if (!streamConfiguration.isPresent()) {
+ return;
+ }
+ ArrayList<Object> ignoredSettings = new ArrayList<Object>();
+ Iterator iterator = streamConfiguration.get().getKeys();
+ while (iterator.hasNext()) {
+ Object setting = iterator.next();
+ if (setting instanceof String && streamSettings.contains(setting)) {
+ String settingStr = (String) setting;
+ setProperty(settingStr, streamConfiguration.get().getProperty(settingStr));
+ } else {
+ ignoredSettings.add(setting);
+ }
+ }
+ if (LOG.isWarnEnabled() && !ignoredSettings.isEmpty()) {
+ LOG.warn("invalid stream configuration override(s): {}",
+ StringUtils.join(ignoredSettings, ";"));
+ }
+ }
+
+ //
+ // ZooKeeper Related Settings
+ //
+
+ /**
+ * Get all properties as a string.
+ */
+ public String getPropsAsString() {
+ Iterator iterator = getKeys();
+ StringBuilder builder = new StringBuilder();
+ boolean appendNewline = false;
+ while (iterator.hasNext()) {
+ Object key = iterator.next();
+ if (key instanceof String) {
+ if (appendNewline) {
+ builder.append("\n");
+ }
+ Object value = getProperty((String)key);
+ builder.append(key).append("=").append(value);
+ appendNewline = true;
+ }
+ }
+ return builder.toString();
+ }
+
+ /**
+ * Get digest id used for ZK acl.
+ *
+ * @return zk acl id.
+ */
+ public String getZkAclId() {
+ return getString(BKDL_ZK_ACL_ID, BKDL_ZK_ACL_ID_DEFAULT);
+ }
+
+ /**
+ * Set digest id to use for ZK acl.
+ *
+ * @param zkAclId acl id.
+ * @return distributedlog configuration
+ * @see #getZkAclId()
+ */
+ public DistributedLogConfiguration setZkAclId(String zkAclId) {
+ setProperty(BKDL_ZK_ACL_ID, zkAclId);
+ return this;
+ }
+
+ /**
+ * Get ZK Session timeout in seconds.
+ * <p>
+ * This is the session timeout applied for zookeeper client used by distributedlog.
+ * Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
+ * by bookkeeper client.
+ *
+ * @return zookeeeper session timeout in seconds.
+ * @deprecated use {@link #getZKSessionTimeoutMilliseconds()}
+ */
+ public int getZKSessionTimeoutSeconds() {
+ return this.getInt(BKDL_ZK_SESSION_TIMEOUT_SECONDS, BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT);
+ }
+
+ /**
+ * Get ZK Session timeout in milliseconds.
+ * <p>
+ * This is the session timeout applied for zookeeper client used by distributedlog.
+ * Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
+ * by bookkeeper client.
+ *
+ * @return zk session timeout in milliseconds.
+ */
+ public int getZKSessionTimeoutMilliseconds() {
+ return this.getInt(BKDL_ZK_SESSION_TIMEOUT_SECONDS, BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT) * 1000;
+ }
+
+ /**
+ * Set ZK Session Timeout in seconds.
+ *
+ * @param zkSessionTimeoutSeconds session timeout in seconds.
+ * @return distributed log configuration
+ * @see #getZKSessionTimeoutMilliseconds()
+ */
+ public DistributedLogConfiguration setZKSessionTimeoutSeconds(int zkSessionTimeoutSeconds) {
+ setProperty(BKDL_ZK_SESSION_TIMEOUT_SECONDS, zkSessionTimeoutSeconds);
+ return this;
+ }
+
+ /**
+ * Get zookeeper access rate limit.
+ * <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
+ * It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
+ * the rate limiting is disable. By default it is disable (value = 0).
+ *
+ * @return zookeeper access rate, by default it is 0.
+ */
+ public double getZKRequestRateLimit() {
+ return this.getDouble(BKDL_ZK_REQUEST_RATE_LIMIT, BKDL_ZK_REQUEST_RATE_LIMIT_DEFAULT);
+ }
+
+ /**
+ * Set zookeeper access rate limit (rps).
+ *
+ * @param requestRateLimit
+ * zookeeper access rate limit
+ * @return distributedlog configuration
+ * @see #getZKRequestRateLimit()
+ */
+ public DistributedLogConfiguration setZKRequestRateLimit(double requestRateLimit) {
+ setProperty(BKDL_ZK_REQUEST_RATE_LIMIT, requestRateLimit);
+ return this;
+ }
+
+ /**
+ * Get num of retries per request for zookeeper client.
+ * <p>Retries only happen on retryable failures like session expired,
+ * session moved. for permanent failures, the request will fail immediately.
+ * The default value is 3.
+ *
+ * @return num of retries per request of zookeeper client.
+ */
+ public int getZKNumRetries() {
+ return this.getInt(BKDL_ZK_NUM_RETRIES, BKDL_ZK_NUM_RETRIES_DEFAULT);
+ }
+
+ /**
+ * Set num of retries per request for zookeeper client.
+ *
+ * @param zkNumRetries num of retries per request of zookeeper client.
+ * @return distributed log configuration
+ * @see #getZKNumRetries()
+ */
+ public DistributedLogConfiguration setZKNumRetries(int zkNumRetries) {
+ setProperty(BKDL_ZK_NUM_RETRIES, zkNumRetries);
+ return this;
+ }
+
+ /**
+ * Get the start backoff time of zookeeper operation retries, in milliseconds.
+ * <p>The retry time will increase in bound exponential way, and become flat
+ * after hit max backoff time ({@link #getZKRetryBackoffMaxMillis()}).
+ * The default start backoff time is 5000 milliseconds.
+ *
+ * @return start backoff time of zookeeper operation retries, in milliseconds.
+ * @see #getZKRetryBackoffMaxMillis()
+ */
+ public int getZKRetryBackoffStartMillis() {
+ return this.getInt(BKDL_ZK_RETRY_BACKOFF_START_MILLIS,
+ BKDL_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT);
+ }
+
+ /**
+ * Set the start backoff time of zookeeper operation retries, in milliseconds.
+ *
+ * @param zkRetryBackoffStartMillis start backoff time of zookeeper operation retries,
+ * in milliseconds.
+ * @return distributed log configuration
+ * @see #getZKRetryBackoffStartMillis()
+ */
+ public DistributedLogConfiguration setZKRetryBackoffStartMillis(int zkRetryBackoffStartMillis) {
+ setProperty(BKDL_ZK_RETRY_BACKOFF_START_MILLIS, zkRetryBackoffStartMillis);
+ return this;
+ }
+
+ /**
+ * Get the max backoff time of zookeeper operation retries, in milliseconds.
+ * <p>The retry time will increase in bound exponential way starting from
+ * {@link #getZKRetryBackoffStartMillis()}, and become flat after hit this max
+ * backoff time.
+ * The default max backoff time is 30000 milliseconds.
+ *
+ * @return max backoff time of zookeeper operation retries, in milliseconds.
+ * @see #getZKRetryBackoffStartMillis()
+ */
+ public int getZKRetryBackoffMaxMillis() {
+ return this.getInt(BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS,
+ BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT);
+ }
+
+ /**
+ * Set the max backoff time of zookeeper operation retries, in milliseconds.
+ *
+ * @param zkRetryBackoffMaxMillis max backoff time of zookeeper operation retries,
+ * in milliseconds.
+ * @return distributed log configuration
+ * @see #getZKRetryBackoffMaxMillis()
+ */
+ public DistributedLogConfiguration setZKRetryBackoffMaxMillis(int zkRetryBackoffMaxMillis) {
+ setProperty(BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS, zkRetryBackoffMaxMillis);
+ return this;
+ }
+
+ /**
+ * Get ZK client number of retry executor threads.
+ * By default it is 1.
+ *
+ * @return number of bookkeeper client worker threads.
+ */
+ public int getZKClientNumberRetryThreads() {
+ return this.getInt(BKDL_ZKCLIENT_NUM_RETRY_THREADS, BKDL_ZKCLIENT_NUM_RETRY_THREADS_DEFAULT);
+ }
+
+ /**
+ * Set ZK client number of retry executor threads.
+ *
+ * @param numThreads
+ * number of retry executor threads.
+ * @return distributedlog configuration.
+ * @see #getZKClientNumberRetryThreads()
+ */
+ public DistributedLogConfiguration setZKClientNumberRetryThreads(int numThreads) {
+ setProperty(BKDL_ZKCLIENT_NUM_RETRY_THREADS, numThreads);
+ return this;
+ }
+
+ //
+ // BookKeeper ZooKeeper Client Settings
+ //
+
+ /**
+ * Get BK's zookeeper session timout in milliseconds.
+ * <p>
+ * This is the session timeout applied for zookeeper client used by bookkeeper client.
+ * Use {@link #getZKSessionTimeoutMilliseconds()} for zookeeper client used
+ * by distributedlog.
+ *
+ * @return Bk's zookeeper session timeout in milliseconds
+ */
+ public int getBKClientZKSessionTimeoutMilliSeconds() {
+ return this.getInt(BKDL_BKCLIENT_ZK_SESSION_TIMEOUT, BKDL_BKCLIENT_ZK_SESSION_TIMEOUT_DEFAULT) * 1000;
+ }
+
+ /**
+ * Set BK's zookeeper session timeout in seconds.
+ *
+ * @param sessionTimeout session timeout for the ZK Client used by BK Client, in seconds.
+ * @return distributed log configuration
+ * @see #getBKClientZKSessionTimeoutMilliSeconds()
+ */
+ public DistributedLogConfiguration setBKClientZKSessionTimeout(int sessionTimeout) {
+ setProperty(BKDL_BKCLIENT_ZK_SESSION_TIMEOUT, sessionTimeout);
+ return this;
+ }
+
+ /**
+ * Get zookeeper access rate limit for zookeeper client used in bookkeeper client.
+ * <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
+ * It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
+ * the rate limiting is disable. By default it is disable (value = 0).
+ *
+ * @return zookeeper access rate limit for zookeeper client used in bookkeeper client.
+ * By default it is 0.
+ */
+ public double getBKClientZKRequestRateLimit() {
+ return this.getDouble(BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT,
+ BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT_DEFAULT);
+ }
+
+ /**
+ * Set zookeeper access rate limit for zookeeper client used in bookkeeper client.
+ *
+ * @param rateLimit
+ * zookeeper access rate limit
+ * @return distributedlog configuration.
+ * @see #getBKClientZKRequestRateLimit()
+ */
+ public DistributedLogConfiguration setBKClientZKRequestRateLimit(double rateLimit) {
+ setProperty(BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT, rateLimit);
+ return this;
+ }
+
+ /**
+ * Get num of retries for zookeeper client that used by bookkeeper client.
+ * <p>Retries only happen on retryable failures like session expired,
+ * session moved. for permanent failures, the request will fail immediately.
+ * The default value is 3. Setting it to zero or negative will retry infinitely.
+ *
+ * @return num of retries of zookeeper client used by bookkeeper client.
+ */
+ public int getBKClientZKNumRetries() {
+ int zkNumRetries = this.getInt(BKDL_BKCLIENT_ZK_NUM_RETRIES, BKDL_BKCLIENT_ZK_NUM_RETRIES_DEFAULT);
+ if (zkNumRetries <= 0) {
+ return Integer.MAX_VALUE;
+ }
+ return zkNumRetries;
+ }
+
+ /**
+ * Get the start backoff time of zookeeper operation retries, in milliseconds.
+ * <p>The retry time will increase in bound exponential way, and become flat
+ * after hit max backoff time ({@link #getBKClientZKRetryBackoffMaxMillis()}.
+ * The default start backoff time is 5000 milliseconds.
+ *
+ * @return start backoff time of zookeeper operation retries, in milliseconds.
+ * @see #getBKClientZKRetryBackoffMaxMillis()
+ */
+ public int getBKClientZKRetryBackoffStartMillis() {
+ return this.getInt(BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS,
+ BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT);
+ }
+
+ /**
+ * Get the max backoff time of zookeeper operation retries, in milliseconds.
+ * <p>The retry time will increase in bound exponential way starting from
+ * {@link #getBKClientZKRetryBackoffStartMillis()}, and become flat after
+ * hit this max backoff time.
+ * The default max backoff time is 30000 milliseconds.
+ *
+ * @return max backoff time of zookeeper operation retries, in milliseconds.
+ * @see #getBKClientZKRetryBackoffStartMillis()
+ */
+ public int getBKClientZKRetryBackoffMaxMillis() {
+ return this.getInt(BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS,
+ BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT);
+ }
+
+ //
+ // BookKeeper Ensemble Placement Settings
+ //
+
+ /**
+ * Get ensemble size of each log segment (ledger) will use.
+ * By default it is 3.
+ * <p>
+ * A log segment's data is stored in an ensemble of bookies in
+ * a stripping way. Each entry will be added in a <code>write-quorum</code>
+ * size of bookies. The add operation will complete once it receives
+ * responses from a <code>ack-quorum</code> size of bookies. The stripping
+ * is done in a round-robin way in bookkeeper.
+ * <p>
+ * For example, we configure the ensemble-size to 5, write-quorum-size to 3,
+ * and ack-quorum-size to 2. The data will be stored in following stripping way.
+ * <pre>
+ * | entry id | bk1 | bk2 | bk3 | bk4 | bk5 |
+ * | 0 | x | x | x | | |
+ * | 1 | | x | x | x | |
+ * | 2 | | | x | x | x |
+ * | 3 | x | | | x | x |
+ * | 4 | x | x | | | x |
+ * | 5 | x | x | x | | |
+ * </pre>
+ * <p>
+ * We don't recommend stripping within a log segment to increase bandwidth.
+ * We'd recommend to strip by `partition` in higher level of distributedlog
+ * to increase performance. so typically the ensemble size will set to be
+ * the same value as write quorum size.
+ *
+ * @return ensemble size
+ * @see #getWriteQuorumSize()
+ * @see #getAckQuorumSize()
+ */
+ public int getEnsembleSize() {
+ return this.getInt(BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
+ getInt(BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
+ BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT));
+ }
+
+ /**
+ * Set ensemble size of each log segment (ledger) will use.
+ *
+ * @param ensembleSize ensemble size.
+ * @return distributed log configuration
+ * @see #getEnsembleSize()
+ */
+ public DistributedLogConfiguration setEnsembleSize(int ensembleSize) {
+ setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE, ensembleSize);
+ return this;
+ }
+
+ /**
+ * Get write quorum size of each log segment (ledger) will use.
+ * By default it is 3.
+ *
+ * @return write quorum size
+ * @see #getEnsembleSize()
+ */
+ public int getWriteQuorumSize() {
+ return this.getInt(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
+ getInt(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
+ BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT));
+ }
+
+ /**
+ * Set write quorum size of each log segment (ledger) will use.
+ *
+ * @param quorumSize
+ * quorum size.
+ * @return distributedlog configuration.
+ * @see #getWriteQuorumSize()
+ */
+ public DistributedLogConfiguration setWriteQuorumSize(int quorumSize) {
+ setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE, quorumSize);
+ return this;
+ }
+
+ /**
+ * Get ack quorum size of each log segment (ledger) will use.
+ * By default it is 2.
+ *
+ * @return ack quorum size
+ * @see #getEnsembleSize()
+ */
+ public int getAckQuorumSize() {
+ return this.getInt(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
+ getInt(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
+ BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT));
+ }
+
+ /**
+ * Set ack quorum size of each log segment (ledger) will use.
+ *
+ * @param quorumSize
+ * quorum size.
+ * @return distributedlog configuration.
+ * @see #getAckQuorumSize()
+ */
+ public DistributedLogConfiguration setAckQuorumSize(int quorumSize) {
+ setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE, quorumSize);
+ return this;
+ }
+
+ /**
+ * Get the quorum config for each log segment (ledger).
+ *
+ * @return quorum config that used by log segments
+ * @see #getEnsembleSize()
+ * @see #getWriteQuorumSize()
+ * @see #getAckQuorumSize()
+ */
+ public QuorumConfig getQuorumConfig() {
+ return new QuorumConfig(
+ getEnsembleSize(),
+ getWriteQuorumSize(),
+ getAckQuorumSize());
+ }
+
+ /**
+ * Get if row aware ensemble placement is enabled.
+ * <p>If enabled, {@link DNSResolverForRows} will be used for dns resolution
+ * rather than {@link DNSResolverForRacks}, if no other dns resolver set via
+ * {@link #setEnsemblePlacementDnsResolverClass(Class)}.
+ * By default it is disable.
+ *
+ * @return true if row aware ensemble placement is enabled, otherwise false.
+ * @see #getEnsemblePlacementDnsResolverClass()
+ */
+ public boolean getRowAwareEnsemblePlacementEnabled() {
+ return getBoolean(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT,
+ getBoolean(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_OLD,
+ BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_DEFAULT));
+ }
+
+ /**
+ * Set if we should enable row aware ensemble placement.
+ *
+ * @param enableRowAwareEnsemblePlacement
+ * enableRowAwareEnsemblePlacement
+ * @return distributedlog configuration.
+ * @see #getRowAwareEnsemblePlacementEnabled()
+ */
+ public DistributedLogConfiguration setRowAwareEnsemblePlacementEnabled(boolean enableRowAwareEnsemblePlacement) {
+ setProperty(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT, enableRowAwareEnsemblePlacement);
+ return this;
+ }
+
+ /**
+ * Get the DNS resolver class for bookkeeper ensemble placement.
+ * <p>By default, {@link DNSResolverForRacks} will be used if
+ * {@link #getRowAwareEnsemblePlacementEnabled()} is disabled and
+ * {@link DNSResolverForRows} will be used if {@link #getRowAwareEnsemblePlacementEnabled()}
+ * is enabled.
+ *
+ * @return dns resolver class for bookkeeper ensemble placement.
+ * @throws ConfigurationException
+ * @see #getRowAwareEnsemblePlacementEnabled()
+ */
+ public Class<? extends DNSToSwitchMapping> getEnsemblePlacementDnsResolverClass()
+ throws ConfigurationException {
+ Class<? extends DNSToSwitchMapping> defaultResolverCls;
+ if (getRowAwareEnsemblePlacementEnabled()) {
+ defaultResolverCls = DNSResolverForRows.class;
+ } else {
+ defaultResolverCls = DNSResolverForRacks.class;
+ }
+ return ReflectionUtils.getClass(this, BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS,
+ defaultResolverCls, DNSToSwitchMapping.class, defaultLoader);
+ }
+
+ /**
+ * Set the DNS resolver class for bookkeeper ensemble placement.
+ *
+ * @param dnsResolverClass
+ * dns resolver class for bookkeeper ensemble placement.
+ * @return distributedlog configuration
+ * @see #getEnsemblePlacementDnsResolverClass()
+ */
+ public DistributedLogConfiguration setEnsemblePlacementDnsResolverClass(
+ Class<? extends DNSToSwitchMapping> dnsResolverClass) {
+ setProperty(BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS, dnsResolverClass.getName());
+ return this;
+ }
+
+ /**
+ * Get mapping used to override the region mapping derived by the default resolver.
+ * <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
+ * By default it is empty string.
+ *
+ * @return dns resolver overrides.
+ * @see #getEnsemblePlacementDnsResolverClass()
+ * @see DNSResolverForRacks
+ * @see DNSResolverForRows
+ */
+ public String getBkDNSResolverOverrides() {
+ return getString(BKDL_BK_DNS_RESOLVER_OVERRIDES, BKDL_BK_DNS_RESOLVER_OVERRIDES_DEFAULT);
+ }
+
+ /**
+ * Set mapping used to override the region mapping derived by the default resolver
+ * <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
+ * By default it is empty string.
+ *
+ * @param overrides
+ * dns resolver overrides
+ * @return dl configuration.
+ * @see #getBkDNSResolverOverrides()
+ */
+ public DistributedLogConfiguration setBkDNSResolverOverrides(String overrides) {
+ setProperty(BKDL_BK_DNS_RESOLVER_OVERRIDES, overrides);
+ return this;
+ }
+
+ //
+ // BookKeeper General Settings
+ //
+
+ /**
+ * Set password used by bookkeeper client for digestion.
+ * <p>
+ * NOTE: not recommend to change. will be derepcated in future.
+ *
+ * @param bkDigestPW BK password digest
+ * @return distributedlog configuration
+ */
+ public DistributedLogConfiguration setBKDigestPW(String bkDigestPW) {
+ setProperty(BKDL_BOOKKEEPER_DIGEST_PW, bkDigestPW);
+ return this;
+ }
+
+ /**
+ * Get password used by bookkeeper client for digestion.
+ * <p>
+ * NOTE: not recommend to change. will be deprecated in future.
+ *
+ * @return password used by bookkeeper client for digestion
+ * @see #setBKDigestPW(String)
+ */
+ public String getBKDigestPW() {
+ return getString(BKDL_BOOKKEEPER_DIGEST_PW, BKDL_BOOKKEEPER_DIGEST_PW_DEFAULT);
+ }
+
+ /**
+ * Get BK client number of i/o threads used by Netty.
+ * The default value equals DL's number worker threads.
+ *
+ * @return number of bookkeeper netty i/o threads.
+ * @see #getNumWorkerThreads()
+ */
+ public int getBKClientNumberIOThreads() {
+ return this.getInt(BKDL_BKCLIENT_NUM_IO_THREADS, getNumWorkerThreads());
+ }
+
+ /**
+ * Set BK client number of i/o threads used by netty.
+ *
+ * @param numThreads
+ * number io threads.
+ * @return distributedlog configuration.
+ * @see #getBKClientNumberIOThreads()
+ */
+ public DistributedLogConfiguration setBKClientNumberIOThreads(int numThreads) {
+ setProperty(BKDL_BKCLIENT_NUM_IO_THREADS, numThreads);
+ return this;
+ }
+
+ /**
+ * Get the tick duration in milliseconds that used for timeout timer in bookkeeper client.
+ * By default it is 100.
+ *
+ * @return tick duration in milliseconds
+ * @see org.jboss.netty.util.HashedWheelTimer
+ */
+ public long getTimeoutTimerTickDurationMs() {
+ return getLong(BKDL_TIMEOUT_TIMER_TICK_DURATION_MS, BKDL_TIMEOUT_TIMER_TICK_DURATION_MS_DEFAULT);
+ }
+
+ /**
+ * Set the tick duration in milliseconds that used for timeout timer in bookkeeper client.
+ *
+ * @param tickDuration
+ * tick duration in milliseconds.
+ * @return distributed log configuration.
+ * @see #getTimeoutTimerTickDurationMs()
+ */
+ public DistributedLogConfiguration setTimeoutTimerTickDurationMs(long tickDuration) {
+ setProperty(BKDL_TIMEOUT_TIMER_TICK_DURATION_MS, tickDuration);
+ return this;
+ }
+
+ /**
+ * Get number of ticks that used for timeout timer in bookkeeper client.
+ * By default is 1024.
+ *
+ * @return number of ticks that used for timeout timer.
+ * @see org.jboss.netty.util.HashedWheelTimer
+ */
+ public int getTimeoutTimerNumTicks() {
+ return getInt(BKDL_TIMEOUT_TIMER_NUM_TICKS, BKDL_TIMEOUT_TIMER_NUM_TICKS_DEFAULT);
+ }
+
+ /**
+ * Set number of ticks that used for timeout timer in bookkeeper client.
+ *
+ * @param numTicks
+ * number of ticks that used for timeout timer.
+ * @return distributed log configuration.
+ * @see #getTimeoutTimerNumTicks()
+ */
+ public DistributedLogConfiguration setTimeoutTimerNumTicks(int numTicks) {
+ setProperty(BKDL_TIMEOUT_TIMER_NUM_TICKS, numTicks);
+ return this;
+ }
+
+ //
+ // Deprecated BookKeeper Settings
+ //
+
+ /**
+ * Get BK client read timeout in seconds.
+ * <p>
+ * Please use {@link ClientConfiguration#getReadEntryTimeout()}
+ * instead of this setting.
+ *
+ * @return read timeout in seconds
+ * @deprecated
+ * @see ClientConfiguration#getReadEntryTimeout()
+ */
+ public int getBKClientReadTimeout() {
+ return this.getInt(BKDL_BKCLIENT_READ_TIMEOUT,
+ BKDL_BKCLIENT_READ_TIMEOUT_DEFAULT);
+ }
+
+ /**
+ * Set BK client read timeout in seconds.
+ *
+ * @param readTimeout read timeout in seconds.
+ * @return distributed log configuration
+ * @deprecated
+ * @see #getBKClientReadTimeout()
+ */
+ public DistributedLogConfiguration setBKClientReadTimeout(int readTimeout) {
+ setProperty(BKDL_BKCLIENT_READ_TIMEOUT, readTimeout);
+ return this;
+ }
+
+ /**
+ * Get BK client write timeout in seconds.
+ * <p>
+ * Please use {@link ClientConfiguration#getAddEntryTimeout()}
+ * instead of this setting.
+ *
+ * @return write timeout in seconds.
+ * @deprecated
+ * @see ClientConfiguration#getAddEntryTimeout()
+ */
+ public int getBKClientWriteTimeout() {
+ return this.getInt(BKDL_BKCLIENT_WRITE_TIMEOUT, BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
+ }
+
+ /**
+ * Set BK client write timeout in seconds
+ *
+ * @param writeTimeout write timeout in seconds.
+ * @return distributed log configuration
+ * @deprecated
+ * @see #getBKClientWriteTimeout()
+ */
+ public DistributedLogConfiguration setBKClientWriteTimeout(int writeTimeout) {
+ setProperty(BKDL_BKCLIENT_WRITE_TIMEOUT, writeTimeout);
+ return this;
+ }
+
+ /**
+ * Get BK client number of worker threads.
+ * <p>
+ * Please use {@link ClientConfiguration#getNumWorkerThreads()}
+ * instead of this setting.
+ *
+ * @return number of bookkeeper client worker threads.
+ * @deprecated
+ * @see ClientConfiguration#getNumWorkerThreads()
+ */
+ public int getBKClientNumberWorkerThreads() {
+ return this.getInt(BKDL_BKCLIENT_NUM_WORKER_THREADS, BKDL_BKCLEINT_NUM_WORKER_THREADS_DEFAULT);
+ }
+
+ /**
+ * Set BK client number of worker threads.
+ *
+ * @param numThreads
+ * number worker threads.
+ * @return distributedlog configuration.
+ * @deprecated
+ * @see #getBKClientNumberWorkerThreads()
+ */
+ public DistributedLogConfiguration setBKClientNumberWorkerThreads(int numThreads) {
+ setProperty(BKDL_BKCLIENT_NUM_WORKER_THREADS, numThreads);
+ return this;
+ }
+
+ //
+ // DL Executor Settings
+ //
+
+ /**
+ * Get the number of worker threads used by distributedlog namespace.
+ * By default it is the number of available processors.
+ *
+ * @return number of worker threads used by distributedlog namespace.
+ */
+ public int getNumWorkerThreads() {
+ return getInt(BKDL_NUM_WORKER_THREADS, Runtime.getRuntime().availableProcessors());
+ }
+
+ /**
+ * Set the number of worker threads used by distributedlog namespace.
+ *
+ * @param numWorkerThreads
+ * number of worker threads used by distributedlog namespace.
+ * @return configuration
+ * @see #getNumWorkerThreads()
+ */
+ public DistributedLogConfiguration setNumWorkerThreads(int numWorkerThreads) {
+ setProperty(BKDL_NUM_WORKER_THREADS, numWorkerThreads);
+ return this;
+ }
+
+ /**
+ * Get the number of dedicated readahead worker threads used by distributedlog namespace.
+ * <p>If this value is non-positive, it would share the normal executor (see {@link #getNumWorkerThreads()}
+ * for readahead. otherwise, it would use a dedicated executor for readhead. By default,
+ * it is 0.
+ *
+ * @return number of dedicated readahead worker threads.
+ * @see #getNumWorkerThreads()
+ */
+ @Deprecated
+ public int getNumReadAheadWorkerThreads() {
+ return getInt(BKDL_NUM_READAHEAD_WORKER_THREADS, 0);
+ }
+
+ /**
+ * Set the number of dedicated readahead worker threads used by distributedlog namespace.
+ *
+ * @param numWorkerThreads
+ * number of dedicated readahead worker threads.
+ * @return configuration
+ * @see #getNumReadAheadWorkerThreads()
+ */
+ @Deprecated
+ public DistributedLogConfiguration setNumReadAheadWorkerThreads(int numWorkerThreads) {
+ setProperty(BKDL_NUM_READAHEAD_WORKER_THREADS, numWorkerThreads);
+ return this;
+ }
+
+ /**
+ * Get the number of lock state threads used by distributedlog namespace.
+ * By default it is 1.
+ *
+ * @return number of lock state threads used by distributedlog namespace.
+ */
+ public int getNumLockStateThreads() {
+ return getInt(BKDL_NUM_LOCKSTATE_THREADS, 1);
+ }
+
+ /**
+ * Set the number of lock state threads used by distributedlog manager factory.
+ *
+ * @param numLockStateThreads
+ * number of lock state threads used by distributedlog manager factory.
+ * @return configuration
+ * @see #getNumLockStateThreads()
+ */
+ public DistributedLogConfiguration setNumLockStateThreads(int numLockStateThreads) {
+ setProperty(BKDL_NUM_LOCKSTATE_THREADS, numLockStateThreads);
+ return this;
+ }
+
+ /**
+ * Get the number of resource release threads used by distributedlog namespace.
+ * By default it is 0 - the thread will be created dynamically by a executor service.
+ * The executor service is an unbounded pool. Application can use `total_tasks - completed_tasks`
+ * on monitoring the number of threads that are used for releasing resources.
+ * <p>
+ * The setting is only applied for v2 implementation.
+ *
+ * @see org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor
+ * @return number of resource release threads used by distributedlog namespace.
+ */
+ public int getNumResourceReleaseThreads() {
+ return getInt(BKDL_NUM_RESOURCE_RELEASE_THREADS, 0);
+ }
+
+ /**
+ * Set the number of resource release threads used by distributedlog manager factory.
+ *
+ * @param numResourceReleaseThreads
+ * number of resource release threads used by distributedlog manager factory.
+ * @return configuration
+ * @see #getNumResourceReleaseThreads()
+ */
+ public DistributedLogConfiguration setNumResourceReleaseThreads(int numResourceReleaseThreads) {
+ setProperty(BKDL_NUM_RESOURCE_RELEASE_THREADS, numResourceReleaseThreads);
+ return this;
+ }
+
+ /**
+ * Get timeout for shutting down schedulers in dl manager, in milliseconds.
+ * By default, it is 5 seconds.
+ *
+ * @return timeout for shutting down schedulers in dl manager, in miliseconds.
+ */
+ public int getSchedulerShutdownTimeoutMs() {
+ return getInt(BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS, BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS_DEFAULT);
+ }
+
+ /**
+ * Set timeout for shutting down schedulers in dl manager, in milliseconds.
+ *
+ * @param timeoutMs
+ * timeout for shutting down schedulers in dl manager, in milliseconds.
+ * @return dl configuration.
+ * @see #getSchedulerShutdownTimeoutMs()
+ */
+ public DistributedLogConfiguration setSchedulerShutdownTimeoutMs(int timeoutMs) {
+ setProperty(BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS, timeoutMs);
+ return this;
+ }
+
+ /**
+ * Whether to use daemon thread for DL threads.
+ * By default it is false.
+ *
+ * @return true if use daemon threads, otherwise false.
+ */
+ public boolean getUseDaemonThread() {
+ return getBoolean(BKDL_USE_DAEMON_THREAD, BKDL_USE_DAEMON_THREAD_DEFAULT);
+ }
+
+ /**
+ * Set whether to use daemon thread for DL threads.
+ *
+ * @param daemon
+ * whether to use daemon thread for DL threads.
+ * @return distributedlog configuration
+ * @see #getUseDaemonThread()
+ */
+ public DistributedLogConfiguration setUseDaemonThread(boolean daemon) {
+ setProperty(BKDL_USE_DAEMON_THREAD, daemon);
+ return this;
+ }
+
+ //
+ // Metadata Settings
+ //
+
+ /**
+ * Get DL ledger metadata output layout version.
+ *
+ * @return layout version
+ * @see org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion
+ */
+ public int getDLLedgerMetadataLayoutVersion() {
+ return this.getInt(BKDL_LEDGER_METADATA_LAYOUT_VERSION,
+ getInt(BKDL_LEDGER_METADATA_LAYOUT_VERSION_OLD,
+ BKDL_LEDGER_METADATA_LAYOUT_VERSION_DEFAULT));
+ }
+
+ /**
+ * Set DL ledger metadata output layout version.
+ *
+ * @param layoutVersion layout version
+ * @return distributed log configuration
+ * @throws IllegalArgumentException if setting an unknown layout version.
+ * @see #getDLLedgerMetadataLayoutVersion()
+ */
+ public DistributedLogConfiguration setDLLedgerMetadataLayoutVersion(int layoutVersion)
+ throws IllegalArgumentException {
+ if ((layoutVersion <= 0) ||
+ (layoutVersion > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION)) {
+ // Incorrect version specified
+ throw new IllegalArgumentException("Incorrect value for ledger metadata layout version");
+ }
+ setProperty(BKDL_LEDGER_METADATA_LAYOUT_VERSION, layoutVersion);
+ return this;
+ }
+
+ /**
+ * Get the setting for whether we should enforce the min ledger metadata version check.
+ * By default it is false.
+ *
+ * @return whether we should enforce the min ledger metadata version check
+ * @see org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion
+ */
+ public boolean getDLLedgerMetadataSkipMinVersionCheck() {
+ return this.getBoolean(BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK,
+ BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK_DEFAULT);
+ }
+
+ /**
+ * Set if we should skip the enforcement of min ledger metadata version.
+ * <p>NOTE: please be aware the side effects of skipping min ledger metadata
+ * version checking.
+ *
+ * @param skipMinVersionCheck whether we should enforce the min ledger metadata version check
+ * @return distributed log configuration
+ * @see #getDLLedgerMetadataSkipMinVersionCheck()
+ */
+ public DistributedLogConfiguration setDLLedgerMetadataSkipMinVersionCheck(boolean skipMinVersionCheck) throws IllegalArgumentException {
+ setProperty(BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK, skipMinVersionCheck);
+ return this;
+ }
+
+ /**
+ * Get the value at which ledger sequence number should start for streams that are being
+ * upgraded and did not have ledger sequence number to start with or for newly created
+ * streams. By default, it is 1.
+ * <p>In most of the cases this value should not be changed. It is useful for backfilling
+ * in the case of migrating log segments whose metadata don't have log segment sequence number.
+ *
+ * @return first ledger sequence number
+ */
+ public long getFirstLogSegmentSequenceNumber() {
+ return this.getLong(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER,
+ getLong(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_OLD,
+ BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_DEFAULT));
+ }
+
+ /**
+ * Set the value at which ledger sequence number should start for streams that are being
+ * upgraded and did not have ledger sequence number to start with or for newly created
+ * streams
+ *
+ * @param firstLogSegmentSequenceNumber first ledger sequence number
+ * @return distributed log configuration
+ * @see #getFirstLogSegmentSequenceNumber()
+ */
+ public DistributedLogConfiguration setFirstLogSegmentSequenceNumber(long firstLogSegmentSequenceNumber)
+ throws IllegalArgumentException {
+ if (firstLogSegmentSequenceNumber <= 0) {
+ // Incorrect ledger sequence number specified
+ throw new IllegalArgumentException("Incorrect value for ledger sequence number");
+ }
+ setProperty(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER, firstLogSegmentSequenceNumber);
+ return this;
+ }
+
+ /**
+ * Whether log segment sequence number validation is enabled?
+ *
+ * @return true if the log segment sequence number validation is enabled, otherwise false.
+ */
+ public boolean isLogSegmentSequenceNumberValidationEnabled() {
+ return this.getBoolean(BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED,
+ BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED_DEFAULT);
+ }
+
+ /**
+ * Whether log segment sequence number validation is enabled?
+ *
+ * @return true if the log segment sequence number validation is enabled, otherwise false.
+ */
+ public DistributedLogConfiguration setLogSegmentSequenceNumberValidationEnabled(boolean enabled) {
+ setProperty(BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED, enabled);
+ return this;
+ }
+
+ /**
+ * Whether we should publish record counts in the log records and metadata.
+ * <p>By default it is true. This is a legacy setting for log segment version 1. It
+ * should be considered removed.
+ *
+ * @return if record counts should be persisted
+ */
+ public boolean getEnableRecordCounts() {
+ return getBoolean(BKDL_ENABLE_RECORD_COUNTS, BKDL_ENABLE_RECORD_COUNTS_DEFAULT);
+ }
+
+ /**
+ * Set if we should publish record counts in the log records and metadata.
+ *
+ * @param enableRecordCounts enable record counts
+ * @return distributed log configuration
+ * @see #getEnableRecordCounts()
+ */
+ public DistributedLogConfiguration setEnableRecordCounts(boolean enableRecordCounts) {
+ setProperty(BKDL_ENABLE_RECORD_COUNTS, enableRecordCounts);
+ return this;
+ }
+
+ /**
+ * Whether sanity check txn id on starting log segments.
+ * <p>If it is enabled, DL writer would throw
+ * {@link org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException}
+ * when it received a smaller transaction id than current maximum transaction id.
+ *
+ * @return true if should check txn id with max txn id, otherwise false.
+ */
+ @Deprecated
+ public boolean getSanityCheckTxnID() {
+ return getBoolean(BKDL_MAXID_SANITYCHECK, BKDL_MAXID_SANITYCHECK_DEFAULT);
+ }
+
+ /**
+ * Enable/Disable sanity check txn id.
+ *
+ * @param enabled
+ * enable/disable sanity check txn id.
+ * @return configuration.
+ * @see #getSanityCheckTxnID()
+ */
+ @Deprecated
+ public DistributedLogConfiguration setSanityCheckTxnID(boolean enabled) {
+ setProperty(BKDL_MAXID_SANITYCHECK, enabled);
+ return this;
+ }
+
+ /**
+ * Whether encode region id in log segment metadata.
+ * <p>In global DL use case, encoding region id in log segment medata would
+ * help understanding what region that a log segment is created. The region
+ * id field in log segment metadata would help for moniotring and troubleshooting.
+ *
+ * @return whether to encode region id in log segment metadata.
+ */
+ public boolean getEncodeRegionIDInLogSegmentMetadata() {
+ return getBoolean(BKDL_ENCODE_REGION_ID_IN_VERSION, BKDL_ENCODE_REGION_ID_IN_VERSION_DEFAULT);
+ }
+
+ /**
+ * Enable/Disable encoding region id in log segment metadata.
+ *
+ * @param enabled
+ * flag to enable/disable encoding region id in log segment metadata.
+ * @return configuration instance.
+ * @see #getEncodeRegionIDInLogSegmentMetadata()
+ */
+ public DistributedLogConfiguration setEncodeRegionIDInLogSegmentMetadata(boolean enabled) {
+ setProperty(BKDL_ENCODE_REGION_ID_IN_VERSION, enabled);
+ return this;
+ }
+
+ /**
+ * Get log segment name version.
+ * <p>
+ * <ul>
+ * <li>version 0: inprogress_(start_txid) |
+ * logrecs_(start_txid)_(end_txid)</li>
+ * <li>version 1: inprogress_(logsegment_sequence_number) |
+ * logrecs_(logsegment_sequence_number)</li>
+ * </ul>
+ * By default it is 1.
+ *
+ * @return log segment name verison.
+ */
+ public int getLogSegmentNameVersion() {
+ return getInt(BKDL_LOGSEGMENT_NAME_VERSION, BKDL_LOGSEGMENT_NAME_VERSION_DEFAULT);
+ }
+
+ /**
+ * Set log segment name version.
+ *
+ * @param version
+ * log segment name version.
+ * @return configuration object.
+ * @see #getLogSegmentNameVersion()
+ */
+ public DistributedLogConfiguration setLogSegmentNameVersion(int version) {
+ setProperty(BKDL_LOGSEGMENT_NAME_VERSION, version);
+ return this;
+ }
+
+ /**
+ * Get name of the unpartitioned stream.
+ * <p>It is a legacy setting. consider removing it in future.
+ *
+ * @return unpartitioned stream
+ */
+ public String getUnpartitionedStreamName() {
+ return getString(BKDL_UNPARTITIONED_STREAM_NAME, BKDL_UNPARTITIONED_STREAM_NAME_DEFAULT);
+ }
+
+ /**
+ * Set name of the unpartitioned stream
+ *
+ * @param streamName name of the unpartitioned stream
+ * @return distributedlog configuration
+ * @see #getUnpartitionedStreamName()
+ */
+ public DistributedLogConfiguration setUnpartitionedStreamName(String streamName) {
+ setProperty(BKDL_UNPARTITIONED_STREAM_NAME, streamName);
+ return this;
+ }
+
+ //
+ // LogSegment Cache Settings
+ //
+
+ /**
+ * Get the log segment cache entry TTL in milliseconds.
+ *
+ * @return log segment cache ttl in milliseconds.
+ */
+ public long getLogSegmentCacheTTLMs() {
+ return getLong(BKDL_LOGSEGMENT_CACHE_TTL_MS, BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT);
+ }
+
+ /**
+ * Set the log segment cache entry TTL in milliseconds.
+ *
+ * @param ttlMs TTL in milliseconds
+ * @return distributedlog configuration
+ */
+ public DistributedLogConfiguration setLogSegmentCacheTTLMs(long ttlMs) {
+ setProperty(BKDL_LOGSEGMENT_CACHE_TTL_MS, ttlMs);
+ return this;
+ }
+
+ /**
+ * Get the maximum size of the log segment cache.
+ *
+ * @return maximum size of the log segment cache.
+ */
+ public long getLogSegmentCacheMaxSize() {
+ return getLong(BKDL_LOGSEGMENT_CACHE_MAX_SIZE, BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT);
+ }
+
+ /**
+ * Set the maximum size of the log segment cache.
+ *
+ * @param maxSize maximum size of the log segment cache.
+ * @return distributedlog configuration
+ */
+ public DistributedLogConfiguration setLogSegmentCacheMaxSize(long maxSize) {
+ setProperty(BKDL_LOGSEGMENT_CACHE_MAX_SIZE, maxSize);
+ return this;
+ }
+
+ /**
+ * Is log segment cache enabled?
+ *
+ * @return true if log segment cache is enabled; otherwise false
+ */
+ public boolean isLogSegmentCacheEnabled() {
+ return getBoolean(BKDL_LOGSEGMENT_CACHE_ENABLED, BKDL_LOGSEGMENT_CACHE_ENABLED_DEFAULT);
+ }
+
+ /**
+ * Enable/disable log segment cache.
+ *
+ * @return distributedlog configuration
+ */
+ public DistributedLogConfiguration setLogSegmentCacheEnabled(boolean enabled) {
+ setProperty(BKDL_LOGSEGMENT_CACHE_ENABLED, enabled);
+ return this;
+ }
+
+ //
+ // DL Writer General Settings
+ //
+
+ /**
+ * Whether to create stream if not exists. By default it is true.
+ *
+ * @return true if it is abled to create stream if not exists.
+ */
+ public boolean getCreateStreamIfNotExists() {
+ return getBoolean(BKDL_CREATE_STREAM_IF_NOT_EXISTS,
+ BKDL_CREATE_STREAM_IF_NOT_EXISTS_DEFAULT);
+ }
+
+ /**
+ * Enable/Disable creating stream if not exists.
+ *
+ * @param enabled
+ * enable/disable sanity check txn id.
+ * @return distributed log configuration.
+ * @see #getCreateStreamIfNotExists()
+ */
+ public DistributedLogConfiguration setCreateStreamIfNotExists(boolean enabled) {
+ setProperty(BKDL_CREATE_STREAM_IF_NOT_EXISTS, enabled);
+ return this;
+ }
+
+ /**
+ * Get Log Flush timeout in seconds.
+ * <p>This is a setting used by DL writer on flushing data. It is typically used
+ * by synchronous writer and log segment writer. By default it is 30 seconds.
+ *
+ * @return log flush timeout in seconds.
+ */
+ // @Deprecated
+ public int getLogFlushTimeoutSeconds() {
+ return this.getInt(BKDL_LOG_FLUSH_TIMEOUT, BKDL_LOG_FLUSH_TIMEOUT_DEFAULT);
+ }
+
+ /**
+ * Set Log Flush Timeout in seconds.
+ *
+ * @param logFlushTimeoutSeconds log flush timeout.
+ * @return distributed log configuration
+ * @see #getLogFlushTimeoutSeconds()
+ */
+ public DistributedLogConfiguration setLogFlushTimeoutSeconds(int logFlushTimeoutSeconds) {
+ setProperty(BKDL_LOG_FLUSH_TIMEOUT, logFlushTimeoutSeconds);
+ return this;
+ }
+
+ /**
+ * The compression type to use while sending data to bookkeeper.
+ *
+ * @return compression type to use
+ * @see org.apache.distributedlog.io.CompressionCodec
+ */
+ public String getCompressionType() {
+ return getString(BKDL_COMPRESSION_TYPE, BKDL_COMPRESSION_TYPE_DEFAULT);
+ }
+
+ /**
+ * Set the compression type to use while sending data to bookkeeper.
+ *
+ * @param compressionType compression type
+ * @return distributedlog configuration
+ * @see #getCompressionType()
+ */
+ public DistributedLogConfiguration setCompressionType(String compressionType) {
+ Preconditions.checkArgument(null != compressionType && !compressionType.isEmpty());
+ setProperty(BKDL_COMPRESSION_TYPE, compressionType);
+ return this;
+ }
+
+ /**
+ * Whether to fail immediately if the stream is not ready rather than queueing the request.
+ * <p>If it is enabled, it would fail the write request immediately if the stream isn't ready.
+ * Consider turning it on for the use cases that could retry writing to other streams
+ * (aka non-strict ordering guarantee). It would result fast failure hence the client would
+ * retry immediately.
+ *
+ * @return true if should fail fast. otherwise, false.
+ */
+ public boolean getFailFastOnStreamNotReady() {
+ return getBoolean(BKDL_FAILFAST_ON_STREAM_NOT_READY,
+ BKDL_FAILFAST_ON_STREAM_NOT_READY_DEFAULT);
+ }
+
+ /**
+ * Set the failfast on stream not ready flag.
+ *
+ * @param failFastOnStreamNotReady
+ * set failfast flag
+ * @return dl configuration.
+ * @see #getFailFastOnStreamNotReady()
+ */
+ public DistributedLogConfiguration setFailFastOnStreamNotReady(boolean failFastOnStreamNotReady) {
+ setProperty(BKDL_FAILFAST_ON_STREAM_NOT_READY, failFastOnStreamNotReady);
+ return this;
+ }
+
+ /**
+ * If this option is set, the log writer won't reset the segment writer if an error
+ * is encountered.
+ *
+ * @return true if we should disable automatic rolling
+ */
+ public boolean getDisableRollingOnLogSegmentError() {
+ return getBoolean(BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR,
+ BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR_DEFAULT);
+ }
+
+ /**
+ * Set the roll on segment error flag.
+ *
+ * @param disableRollingOnLogSegmentError
+ * set roll on error flag
+ * @return dl configuration.
+ * @see #getDisableRollingOnLogSegmentError()
+ */
+ public DistributedLogConfiguration setDisableRollingOnLogSegmentError(boolean disableRollingOnLogSegmentError) {
+ setProperty(BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR, disableRollingOnLogSegmentError);
+ return this;
+ }
+
+ //
+ // DL Durability Settings
+ //
+
+ /**
+ * Check whether the durable write is enabled.
+ * <p>It is enabled by default.
+ *
+ * @return true if durable write is enabled. otherwise, false.
+ */
+ public boolean isDurableWriteEnabled() {
+ return this.getBoolean(BKDL_IS_DURABLE_WRITE_ENABLED, BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT);
+ }
+
+ /**
+ * Enable/Disable durable writes in writers.
+ *
+ * @param enabled
+ * flag to enable/disable durable writes in writers.
+ * @return distributedlog configuration
+ */
+ public DistributedLogConfiguration setDurableWriteEnabled(boolean enabled) {
+ setProperty(BKDL_IS_DURABLE_WRITE_ENABLED, enabled);
+ return this;
+ }
+
+ //
+ // DL Writer Transmit Settings
+ //
+
+ /**
+ * Get output buffer size for DL writers, in bytes.
+ * <p>Large buffer will result in higher compression ratio and
+ * it would use the bandwidth more efficiently and improve throughput.
+ * Set it to 0 would ask DL writers to transmit the data immediately,
+ * which it could achieve low latency.
+ * <p>The default value is 1KB.
+ *
+ * @return buffer size in byes.
+ */
+ public int getOutputBufferSize() {
+ return this.getInt(BKDL_OUTPUT_BUFFER_SIZE,
+ getInt(BKDL_OUTPUT_BUFFER_SIZE_OLD, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT));
+ }
+
+ /**
+ * Set output buffer size for DL writers, in bytes.
+ *
+ * @param opBufferSize output buffer size.
+ * @return distributed log configuration
+ * @see #getOutputBufferSize()
+ */
+ public DistributedLogConfiguration setOutputBufferSize(int opBufferSize) {
+ setProperty(BKDL_OUTPUT_BUFFER_SIZE, opBufferSize);
+ return this;
+ }
+
+ /**
+ * Get Periodic Log Flush Frequency in milliseconds.
+ * <p>If the setting is set with a positive value, the data in output buffer
+ * will be flushed in this provided interval. The default value is 0.
+ *
+ * @return periodic flush frequency in milliseconds.
+ * @see #getOutputBufferSize()
+ */
+ public int getPeriodicFlushFrequencyMilliSeconds() {
+ return this.getInt(BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
+ BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT);
+ }
+
+ /**
+ * Set Periodic Log Flush Frequency in milliseconds.
+ *
+ * @param flushFrequencyMs periodic flush frequency in milliseconds.
+ * @return distributed log configuration
+ * @see #getPeriodicFlushFrequencyMilliSeconds()
+ */
+ public DistributedLogConfiguration setPeriodicFlushFrequencyMilliSeconds(int flushFrequencyMs) {
+ setProperty(BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS, flushFrequencyMs);
+ return this;
+ }
+
+ /**
+ * Is immediate flush enabled.
+ * <p>If it is enabled, it would flush control record immediately after adding
+ * data completed. The default value is false.
+ *
+ * @return whether immediate flush is enabled
+ */
+ public boolean getImmediateFlushEnabled() {
+ return getBoolean(BKDL_ENABLE_IMMEDIATE_FLUSH, BKDL_ENABLE_IMMEDIATE_FLUSH_DEFAULT);
+ }
+
+ /**
+ * Enable/Disable immediate flush
+ *
+ * @param enabled
+ * flag to enable/disable immediate flush.
+ * @return configuration instance.
+ * @see #getImmediateFlushEnabled()
+ */
+ public DistributedLogConfiguration setImmediateFlushEnabled(boolean enabled) {
+ setProperty(BKDL_ENABLE_IMMEDIATE_FLUSH, enabled);
+ return this;
+ }
+
+ /**
+ * Get minimum delay between immediate flushes in milliseconds.
+ * <p>This setting only takes effects when {@link #getImmediateFlushEnabled()}
+ * is enabled. It torelants the bursty of traffic when immediate flush is enabled,
+ * which prevents sending too many control records to the bookkeeper.
+ *
+ * @return minimum delay between immediate flushes in milliseconds
+ * @see #getImmediateFlushEnabled()
+ */
+ public int getMinDelayBetweenImmediateFlushMs() {
+ return this.getInt(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS, BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT);
+ }
+
+ /**
+ * Set minimum delay between immediate flushes in milliseconds
+ *
+ * @param minDelayMs minimum delay between immediate flushes in milliseconds.
+ * @return distributed log configuration
+ * @see #getMinDelayBetweenImmediateFlushMs()
+ */
+ public DistributedLogConfiguration setMinDelayBetweenImmediateFlushMs(int minDelayMs) {
+ setProperty(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS, minDelayMs);
+ return this;
+ }
+
+ /**
+ * Get Periodic Keep Alive Frequency in milliseconds.
+ * <p>If the setting is set with a positive value, it would periodically write a control record
+ * to keep the stream active. The default value is 0.
+ *
+ * @return periodic keep alive frequency in milliseconds.
+ */
+ public int getPeriodicKeepAliveMilliSeconds() {
+ return this.getInt(BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS, BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS_DEFAULT);
+ }
+
+ /**
+ * Set Periodic Keep Alive Frequency in milliseconds.
+ *
+ * @param keepAliveMs keep alive frequency in milliseconds.
+ * @return distributedlog configuration
+ * @see #getPeriodicKeepAliveMilliSeconds()
+ */
+ public DistributedLogConfiguration setPeriodicKeepAliveMilliSeconds(int keepAliveMs) {
+ setProperty(BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS, keepAliveMs);
+ return this;
+ }
+
+ //
+ // DL Retention/Truncation Settings
+ //
+
+ /**
+ * Get log segment retention period in hours.
+ * The default value is 3 days.
+ *
+ * @return log segment retention period in hours
+ */
+ public int getRetentionPeriodHours() {
+ return this.getInt(BKDL_RETENTION_PERIOD_IN_HOURS,
+ getInt(BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
+ BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT));
+ }
+
+ /**
+ * Set log segment retention period in hours.
+ *
+ * @param retentionHours retention period in hours.
+ * @return distributed log configuration
+ */
+ public DistributedLogConfiguration setRetentionPeriodHours(int retentionHours) {
+ setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, retentionHours);
+ return this;
+ }
+
+ /**
+ * Is truncation managed explicitly by the application.
+ * <p>If this is set then time based retention is only a hint to perform
+ * deferred cleanup. However we never remove a segment that has not been
+ * already marked truncated.
+ * <p>It is disabled by default.
+ *
+ * @return whether truncation managed explicitly by the application
+ * @see org.apache.distributedlog.LogSegmentMetadata.TruncationStatus
+ */
+ public boolean getExplicitTruncationByApplication() {
+ return getBoolean(BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION,
+ BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION_DEFAULT);
+ }
+
+ /**
+ * Enable/Disable whether truncation is managed explicitly by the application.
+ *
+ * @param enabled
+ * flag to enable/disable whether truncation is managed explicitly by the application.
+ * @return configuration instance.
+ */
+ public DistributedLogConfiguration setExplicitTruncationByApplication(boolean enabled) {
+ setProperty(BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION, enabled);
+ return this;
+ }
+
+ //
+ // Log Segment Rolling Settings
+ //
+
+ /**
+ * Get log segment rolling interval in minutes.
+ * <p>If the setting is set to a positive value, DL writer will roll log segments
+ * based on time. Otherwise, it will roll log segments based on size.
+ * <p>The default value is 2 hours.
+ *
+ * @return log segment rolling interval in minutes
+ * @see #getMaxLogS
<TRUNCATED>
[49/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 072c3ef..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java
+++ /dev/null
@@ -1,151 +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.benchmark.stream;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.BookKeeperClientBuilder;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Benchmark ledger reading.
- */
-public class LedgerReadBenchmark extends AbstractReaderBenchmark {
-
- private static final Logger logger = LoggerFactory.getLogger(AsyncReaderBenchmark.class);
-
- @Override
- protected void benchmark(DistributedLogNamespace namespace, String logName, StatsLogger statsLogger) {
- DistributedLogManager dlm = null;
- while (null == dlm) {
- try {
- dlm = namespace.openLog(streamName);
- } catch (IOException ioe) {
- logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
- }
- if (null == dlm) {
- try {
- TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
- } catch (InterruptedException e) {
- logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
- streamName, e);
- }
- }
- }
- logger.info("Created dlm for stream {}.", streamName);
-
- List<LogSegmentMetadata> segments = null;
- while (null == segments) {
- try {
- segments = dlm.getLogSegments();
- } catch (IOException ioe) {
- logger.warn("Failed to get log segments for stream {} : ", streamName, ioe);
- }
- if (null == segments) {
- try {
- TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
- } catch (InterruptedException e) {
- logger.warn("Interrupted from sleep while geting log segments for stream {} : ",
- streamName, e);
- }
- }
- }
-
- final Counter readCounter = statsLogger.getCounter("reads");
-
- logger.info("Reading from log segments : {}", segments);
-
- ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
- .uri(uri)
- .name("benchmark-zkc")
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .zkAclId(null)
- .build();
- BKDLConfig bkdlConfig;
- try {
- bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
- } catch (IOException e) {
- return;
- }
-
- BookKeeper bk;
- try {
- bk = BookKeeperClientBuilder.newBuilder()
- .name("benchmark-bkc")
- .dlConfig(conf)
- .zkServers(bkdlConfig.getBkZkServersForReader())
- .ledgersPath(bkdlConfig.getBkLedgersPath())
- .build()
- .get();
- } catch (IOException e) {
- return;
- }
-
- final int readConcurrency = conf.getInt("ledger_read_concurrency", 1000);
- boolean streamRead = conf.getBoolean("ledger_stream_read", true);
- try {
- for (LogSegmentMetadata segment : segments) {
- Stopwatch stopwatch = Stopwatch.createStarted();
- long lid = segment.getLogSegmentId();
- LedgerHandle lh = bk.openLedgerNoRecovery(
- lid, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
- logger.info("It took {} ms to open log segment {}",
- new Object[] { stopwatch.elapsed(TimeUnit.MILLISECONDS), (lh.getLastAddConfirmed() + 1), segment });
- stopwatch.reset().start();
- Runnable reader;
- if (streamRead) {
- reader = new LedgerStreamReader(lh, new BookkeeperInternalCallbacks.ReadEntryListener() {
- @Override
- public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object ctx) {
- readCounter.inc();
- }
- }, readConcurrency);
- } else {
- reader = new LedgerStreamReader(lh, new BookkeeperInternalCallbacks.ReadEntryListener() {
- @Override
- public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object ctx) {
- readCounter.inc();
- }
- }, readConcurrency);
- }
- reader.run();
- logger.info("It took {} ms to complete reading {} entries from log segment {}",
- new Object[] { stopwatch.elapsed(TimeUnit.MILLISECONDS), (lh.getLastAddConfirmed() + 1), segment });
- }
- } catch (Exception e) {
- logger.error("Error on reading bk ", e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerStreamReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerStreamReader.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerStreamReader.java
deleted file mode 100644
index e542af7..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerStreamReader.java
+++ /dev/null
@@ -1,131 +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.benchmark.stream;
-
-import java.util.Enumeration;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Reading ledger in a streaming way.
- */
-public class LedgerStreamReader implements Runnable {
-
- private static final Logger logger = LoggerFactory.getLogger(LedgerStreamReader.class);
-
- class PendingReadRequest implements AsyncCallback.ReadCallback {
-
- final long entryId;
- boolean isDone = false;
- int rc;
- LedgerEntry entry = null;
-
- PendingReadRequest(long entryId) {
- this.entryId = entryId;
- }
-
- void read() {
- lh.asyncReadEntries(entryId, entryId, this, null);
- }
-
- void complete(ReadEntryListener listener) {
- listener.onEntryComplete(rc, lh, entry, null);
- }
-
- @Override
- public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> enumeration, Object ctx) {
- this.rc = rc;
- if (BKException.Code.OK == rc && enumeration.hasMoreElements()) {
- entry = enumeration.nextElement();
- } else {
- entry = null;
- }
- isDone = true;
- // construct a new read request
- long nextEntry = nextReadEntry.getAndIncrement();
- if (nextEntry <= lac) {
- PendingReadRequest nextRead =
- new PendingReadRequest(nextEntry);
- pendingReads.add(nextRead);
- nextRead.read();
- }
- triggerCallbacks();
- }
- }
-
- private final LedgerHandle lh;
- private final long lac;
- private final ReadEntryListener readEntryListener;
- private final int concurrency;
- private final AtomicLong nextReadEntry = new AtomicLong(0);
- private final CountDownLatch done = new CountDownLatch(1);
- private final ConcurrentLinkedQueue<PendingReadRequest> pendingReads =
- new ConcurrentLinkedQueue<PendingReadRequest>();
-
- public LedgerStreamReader(LedgerHandle lh,
- ReadEntryListener readEntryListener,
- int concurrency) {
- this.lh = lh;
- this.lac = lh.getLastAddConfirmed();
- this.readEntryListener = readEntryListener;
- this.concurrency = concurrency;
- for (int i = 0; i < concurrency; i++) {
- long entryId = nextReadEntry.getAndIncrement();
- if (entryId > lac) {
- break;
- }
- PendingReadRequest request = new PendingReadRequest(entryId);
- pendingReads.add(request);
- request.read();
- }
- if (pendingReads.isEmpty()) {
- done.countDown();
- }
- }
-
- synchronized void triggerCallbacks() {
- PendingReadRequest request;
- while ((request = pendingReads.peek()) != null) {
- if (!request.isDone) {
- break;
- }
- pendingReads.remove();
- request.complete(readEntryListener);
- }
- if (pendingReads.isEmpty()) {
- done.countDown();
- }
- }
-
- @Override
- public void run() {
- try {
- done.await();
- } catch (InterruptedException e) {
- logger.info("Interrupted on stream reading ledger {} : ", lh.getId(), e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/ReadMode.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/ReadMode.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/ReadMode.java
deleted file mode 100644
index 280c9db..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/ReadMode.java
+++ /dev/null
@@ -1,28 +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.benchmark.stream;
-
-/**
- * The read mode for streaming read benchmark.
- */
-public enum ReadMode {
- OLDEST,
- LATEST,
- REWIND,
- POSITION
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/StreamBenchmark.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/StreamBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/StreamBenchmark.java
deleted file mode 100644
index 1eff65a..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/StreamBenchmark.java
+++ /dev/null
@@ -1,138 +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.benchmark.stream;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import java.io.File;
-import java.net.URI;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Benchmark Streams.
- */
-public abstract class StreamBenchmark {
-
- private static final Logger logger = LoggerFactory.getLogger(StreamBenchmark.class);
-
- private static final String USAGE = "StreamBenchmark <benchmark-class> [options]";
-
- protected final Options options = new Options();
- protected URI uri;
- protected DistributedLogConfiguration conf;
- protected StatsProvider statsProvider;
- protected String streamName;
-
- protected StreamBenchmark() {
- options.addOption("c", "conf", true, "Configuration File");
- options.addOption("u", "uri", true, "DistributedLog URI");
- options.addOption("p", "stats-provider", true, "Stats Provider");
- options.addOption("s", "stream", true, "Stream Name");
- options.addOption("h", "help", false, "Print usage.");
- }
-
- protected Options getOptions() {
- return options;
- }
-
- protected void printUsage() {
- HelpFormatter hf = new HelpFormatter();
- hf.printHelp(USAGE, options);
- }
-
- protected void parseCommandLine(String[] args)
- throws Exception {
- BasicParser parser = new BasicParser();
- CommandLine cmdline = parser.parse(options, args);
- if (cmdline.hasOption("h")) {
- printUsage();
- System.exit(0);
- }
- if (cmdline.hasOption("u")) {
- this.uri = URI.create(cmdline.getOptionValue("u"));
- } else {
- printUsage();
- System.exit(0);
- }
- this.conf = new DistributedLogConfiguration();
- if (cmdline.hasOption("c")) {
- String configFile = cmdline.getOptionValue("c");
- this.conf.loadConf(new File(configFile).toURI().toURL());
- }
- if (cmdline.hasOption("p")) {
- statsProvider = ReflectionUtils.newInstance(cmdline.getOptionValue("p"), StatsProvider.class);
- } else {
- statsProvider = new NullStatsProvider();
- }
- if (cmdline.hasOption("s")) {
- this.streamName = cmdline.getOptionValue("s");
- } else {
- printUsage();
- System.exit(0);
- }
- parseCommandLine(cmdline);
- }
-
- protected abstract void parseCommandLine(CommandLine cmdline);
-
- protected void run(String[] args) throws Exception {
- logger.info("Parsing arguments for benchmark : {}", args);
- // parse command line
- parseCommandLine(args);
- statsProvider.start(conf);
- // run the benchmark
- StatsLogger statsLogger = statsProvider.getStatsLogger("dl");
- DistributedLogNamespace namespace =
- DistributedLogNamespaceBuilder.newBuilder()
- .conf(conf)
- .uri(uri)
- .statsLogger(statsLogger)
- .build();
- try {
- benchmark(namespace, streamName, statsProvider.getStatsLogger("benchmark"));
- } finally {
- namespace.close();
- statsProvider.stop();
- }
- }
-
- protected abstract void benchmark(DistributedLogNamespace namespace,
- String logName,
- StatsLogger statsLogger);
-
- public static void main(String[] args) throws Exception {
- if (args.length <= 0) {
- System.err.println(USAGE);
- return;
- }
- String benchmarkClassName = args[0];
- StreamBenchmark benchmark = ReflectionUtils.newInstance(
- benchmarkClassName, StreamBenchmark.class);
- benchmark.run(args);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/SyncReaderBenchmark.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/SyncReaderBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/SyncReaderBenchmark.java
deleted file mode 100644
index 122c8ef..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/SyncReaderBenchmark.java
+++ /dev/null
@@ -1,164 +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.benchmark.stream;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogReader;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Benchmark on {@link com.twitter.distributedlog.LogReader} reading from a stream.
- */
-public class SyncReaderBenchmark extends AbstractReaderBenchmark {
-
- private static final Logger logger = LoggerFactory.getLogger(SyncReaderBenchmark.class);
-
- public SyncReaderBenchmark() {}
-
- @Override
- protected void benchmark(DistributedLogNamespace namespace, String streamName, StatsLogger statsLogger) {
- DistributedLogManager dlm = null;
- while (null == dlm) {
- try {
- dlm = namespace.openLog(streamName);
- } catch (IOException ioe) {
- logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
- }
- if (null == dlm) {
- try {
- TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
- } catch (InterruptedException e) {
- logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
- streamName, e);
- }
- }
- }
- OpStatsLogger openReaderStats = statsLogger.getOpStatsLogger("open_reader");
- OpStatsLogger nonBlockingReadStats = statsLogger.getOpStatsLogger("non_blocking_read");
- OpStatsLogger blockingReadStats = statsLogger.getOpStatsLogger("blocking_read");
- Counter nullReadCounter = statsLogger.getCounter("null_read");
-
- logger.info("Created dlm for stream {}.", streamName);
- LogReader reader = null;
- Long lastTxId = null;
- while (null == reader) {
- // initialize the last txid
- if (null == lastTxId) {
- switch (readMode) {
- case OLDEST:
- lastTxId = 0L;
- break;
- case LATEST:
- try {
- lastTxId = dlm.getLastTxId();
- } catch (IOException ioe) {
- continue;
- }
- break;
- case REWIND:
- lastTxId = System.currentTimeMillis() - rewindMs;
- break;
- case POSITION:
- lastTxId = fromTxId;
- break;
- default:
- logger.warn("Unsupported mode {}", readMode);
- printUsage();
- System.exit(0);
- break;
- }
- logger.info("Reading from transaction id {}", lastTxId);
- }
- // Open the reader
- Stopwatch stopwatch = Stopwatch.createStarted();
- try {
- reader = dlm.getInputStream(lastTxId);
- long elapsedMs = stopwatch.elapsed(TimeUnit.MICROSECONDS);
- openReaderStats.registerSuccessfulEvent(elapsedMs);
- logger.info("It took {} ms to position the reader to transaction id {}", lastTxId);
- } catch (IOException ioe) {
- openReaderStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
- logger.warn("Failed to create reader for stream {} reading from {}.", streamName, lastTxId);
- }
- if (null == reader) {
- try {
- TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
- } catch (InterruptedException e) {
- logger.warn("Interrupted from sleep after reader was reassigned null for stream {} : ",
- streamName, e);
- }
- continue;
- }
-
- // read loop
-
- LogRecord record;
- boolean nonBlocking = false;
- stopwatch = Stopwatch.createUnstarted();
- long numCatchupReads = 0L;
- long numCatchupBytes = 0L;
- Stopwatch catchupStopwatch = Stopwatch.createStarted();
- while (true) {
- try {
- stopwatch.start();
- record = reader.readNext(nonBlocking);
- if (null != record) {
- long elapsedMicros = stopwatch.stop().elapsed(TimeUnit.MICROSECONDS);
- if (nonBlocking) {
- nonBlockingReadStats.registerSuccessfulEvent(elapsedMicros);
- } else {
- numCatchupBytes += record.getPayload().length;
- ++numCatchupReads;
- blockingReadStats.registerSuccessfulEvent(elapsedMicros);
- }
- lastTxId = record.getTransactionId();
- } else {
- nullReadCounter.inc();
- }
- if (null == record && !nonBlocking) {
- nonBlocking = true;
- catchupStopwatch.stop();
- logger.info("Catchup {} records (total {} bytes) in {} milliseconds",
- new Object[] { numCatchupReads, numCatchupBytes,
- stopwatch.elapsed(TimeUnit.MILLISECONDS) });
- }
- stopwatch.reset();
- } catch (IOException e) {
- logger.warn("Encountered reading record from stream {} : ", streamName, e);
- reader = null;
- break;
- }
- }
- try {
- TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
- } catch (InterruptedException e) {
- logger.warn("Interrupted from sleep while creating reader for stream {} : ",
- streamName, e);
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/package-info.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/package-info.java
deleted file mode 100644
index d8e198c..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Stream level benchmarks.
- */
-package com.twitter.distributedlog.benchmark.stream;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/ShiftableRateLimiter.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/ShiftableRateLimiter.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/ShiftableRateLimiter.java
deleted file mode 100644
index def0346..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/ShiftableRateLimiter.java
+++ /dev/null
@@ -1,71 +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.benchmark.utils;
-
-import com.google.common.util.concurrent.RateLimiter;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-/**
- * A wrapper over rate limiter.
- */
-public class ShiftableRateLimiter implements Runnable {
-
- private final RateLimiter rateLimiter;
- private final ScheduledExecutorService executor;
- private final double initialRate, maxRate, changeRate;
- private final long changeInterval;
- private final TimeUnit changeIntervalUnit;
- private double nextRate;
-
- public ShiftableRateLimiter(double initialRate,
- double maxRate,
- double changeRate,
- long changeInterval,
- TimeUnit changeIntervalUnit) {
- this.initialRate = initialRate;
- this.maxRate = maxRate;
- this.changeRate = changeRate;
- this.nextRate = initialRate;
- this.changeInterval = changeInterval;
- this.changeIntervalUnit = changeIntervalUnit;
- this.rateLimiter = RateLimiter.create(initialRate);
- this.executor = Executors.newSingleThreadScheduledExecutor();
- this.executor.scheduleAtFixedRate(this, changeInterval, changeInterval, changeIntervalUnit);
- }
-
- public ShiftableRateLimiter duplicate() {
- return new ShiftableRateLimiter(
- initialRate,
- maxRate,
- changeRate,
- changeInterval,
- changeIntervalUnit);
- }
-
- @Override
- public void run() {
- this.nextRate = Math.min(nextRate + changeRate, maxRate);
- this.rateLimiter.setRate(nextRate);
- }
-
- public RateLimiter getLimiter() {
- return this.rateLimiter;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/package-info.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/package-info.java
deleted file mode 100644
index 369b979..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Utils for benchmarking.
- */
-package com.twitter.distributedlog.benchmark.utils;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Benchmarker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Benchmarker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Benchmarker.java
new file mode 100644
index 0000000..f724102
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Benchmarker.java
@@ -0,0 +1,468 @@
+/**
+ * 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 org.apache.distributedlog.benchmark;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.benchmark.utils.ShiftableRateLimiter;
+import com.twitter.finagle.stats.OstrichStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The launcher for benchmarks.
+ */
+public class Benchmarker {
+
+ private static final Logger logger = LoggerFactory.getLogger(Benchmarker.class);
+
+ static final String USAGE = "Benchmarker [-u <uri>] [-c <conf>] [-s serverset] [-m (read|write|dlwrite)]";
+
+ final String[] args;
+ final Options options = new Options();
+
+ int rate = 100;
+ int maxRate = 1000;
+ int changeRate = 100;
+ int changeRateSeconds = 1800;
+ int concurrency = 10;
+ String streamPrefix = "dlog-loadtest";
+ int shardId = -1;
+ int numStreams = 10;
+ List<String> serversetPaths = new ArrayList<String>();
+ List<String> finagleNames = new ArrayList<String>();
+ int msgSize = 256;
+ String mode = null;
+ int durationMins = 60;
+ URI dlUri = null;
+ int batchSize = 0;
+ int readersPerStream = 1;
+ Integer maxStreamId = null;
+ int truncationInterval = 3600;
+ Integer startStreamId = null;
+ Integer endStreamId = null;
+ int hostConnectionCoreSize = 10;
+ int hostConnectionLimit = 10;
+ boolean thriftmux = false;
+ boolean handshakeWithClientInfo = false;
+ boolean readFromHead = false;
+ int sendBufferSize = 1024 * 1024;
+ int recvBufferSize = 1024 * 1024;
+ boolean enableBatching = false;
+ int batchBufferSize = 256 * 1024;
+ int batchFlushIntervalMicros = 2000;
+ String routingServiceFinagleNameString;
+
+ final DistributedLogConfiguration conf = new DistributedLogConfiguration();
+ final StatsReceiver statsReceiver = new OstrichStatsReceiver();
+ StatsProvider statsProvider = null;
+
+ Benchmarker(String[] args) {
+ this.args = args;
+ // prepare options
+ options.addOption("s", "serverset", true, "Proxy Server Set (separated by ',')");
+ options.addOption("fn", "finagle-name", true, "Write proxy finagle name (separated by ',')");
+ options.addOption("c", "conf", true, "DistributedLog Configuration File");
+ options.addOption("u", "uri", true, "DistributedLog URI");
+ options.addOption("i", "shard", true, "Shard Id");
+ options.addOption("p", "provider", true, "DistributedLog Stats Provider");
+ options.addOption("d", "duration", true, "Duration (minutes)");
+ options.addOption("sp", "streamprefix", true, "Stream Prefix");
+ options.addOption("sc", "streamcount", true, "Number of Streams");
+ options.addOption("ms", "messagesize", true, "Message Size (bytes)");
+ options.addOption("bs", "batchsize", true, "Batch Size");
+ options.addOption("r", "rate", true, "Rate limit (requests/second)");
+ options.addOption("mr", "max-rate", true, "Maximum Rate limit (requests/second)");
+ options.addOption("cr", "change-rate", true, "Rate to increase each change period (requests/second)");
+ options.addOption("ci", "change-interval", true, "Rate to increase period, seconds");
+ options.addOption("t", "concurrency", true, "Concurrency (number of threads)");
+ options.addOption("m", "mode", true, "Benchmark mode (read/write)");
+ options.addOption("rps", "readers-per-stream", true, "Number readers per stream");
+ options.addOption("msid", "max-stream-id", true, "Max Stream ID");
+ options.addOption("ti", "truncation-interval", true, "Truncation interval in seconds");
+ options.addOption("ssid", "start-stream-id", true, "Start Stream ID");
+ options.addOption("esid", "end-stream-id", true, "Start Stream ID");
+ options.addOption("hccs", "host-connection-core-size", true, "Finagle hostConnectionCoreSize");
+ options.addOption("hcl", "host-connection-limit", true, "Finagle hostConnectionLimit");
+ options.addOption("mx", "thriftmux", false, "Enable thriftmux (write mode only)");
+ options.addOption("hsci", "handshake-with-client-info", false, "Enable handshaking with client info");
+ options.addOption("rfh", "read-from-head", false, "Read from head of the stream");
+ options.addOption("sb", "send-buffer", true, "Channel send buffer size, in bytes");
+ options.addOption("rb", "recv-buffer", true, "Channel recv buffer size, in bytes");
+ options.addOption("bt", "enable-batch", false, "Enable batching on writers");
+ options.addOption("bbs", "batch-buffer-size", true, "The batch buffer size in bytes");
+ options.addOption("bfi", "batch-flush-interval", true, "The batch buffer flush interval in micros");
+ options.addOption("rs", "routing-service", true, "The routing service finagle name for server-side routing");
+ options.addOption("h", "help", false, "Print usage.");
+ }
+
+ void printUsage() {
+ HelpFormatter helpFormatter = new HelpFormatter();
+ helpFormatter.printHelp(USAGE, options);
+ }
+
+ void run() throws Exception {
+ logger.info("Running benchmark.");
+
+ BasicParser parser = new BasicParser();
+ CommandLine cmdline = parser.parse(options, args);
+ if (cmdline.hasOption("h")) {
+ printUsage();
+ System.exit(0);
+ }
+ if (cmdline.hasOption("s")) {
+ String serversetPathStr = cmdline.getOptionValue("s");
+ serversetPaths = Arrays.asList(StringUtils.split(serversetPathStr, ','));
+ }
+ if (cmdline.hasOption("fn")) {
+ String finagleNameStr = cmdline.getOptionValue("fn");
+ finagleNames = Arrays.asList(StringUtils.split(finagleNameStr, ','));
+ }
+ if (cmdline.hasOption("i")) {
+ shardId = Integer.parseInt(cmdline.getOptionValue("i"));
+ }
+ if (cmdline.hasOption("d")) {
+ durationMins = Integer.parseInt(cmdline.getOptionValue("d"));
+ }
+ if (cmdline.hasOption("sp")) {
+ streamPrefix = cmdline.getOptionValue("sp");
+ }
+ if (cmdline.hasOption("sc")) {
+ numStreams = Integer.parseInt(cmdline.getOptionValue("sc"));
+ }
+ if (cmdline.hasOption("ms")) {
+ msgSize = Integer.parseInt(cmdline.getOptionValue("ms"));
+ }
+ if (cmdline.hasOption("r")) {
+ rate = Integer.parseInt(cmdline.getOptionValue("r"));
+ }
+ if (cmdline.hasOption("mr")) {
+ maxRate = Integer.parseInt(cmdline.getOptionValue("mr"));
+ }
+ if (cmdline.hasOption("cr")) {
+ changeRate = Integer.parseInt(cmdline.getOptionValue("cr"));
+ }
+ if (cmdline.hasOption("ci")) {
+ changeRateSeconds = Integer.parseInt(cmdline.getOptionValue("ci"));
+ }
+ if (cmdline.hasOption("t")) {
+ concurrency = Integer.parseInt(cmdline.getOptionValue("t"));
+ }
+ if (cmdline.hasOption("m")) {
+ mode = cmdline.getOptionValue("m");
+ }
+ if (cmdline.hasOption("u")) {
+ dlUri = URI.create(cmdline.getOptionValue("u"));
+ }
+ if (cmdline.hasOption("bs")) {
+ batchSize = Integer.parseInt(cmdline.getOptionValue("bs"));
+ checkArgument("write" != mode, "batchSize supported only for mode=write");
+ }
+ if (cmdline.hasOption("c")) {
+ String configFile = cmdline.getOptionValue("c");
+ conf.loadConf(new File(configFile).toURI().toURL());
+ }
+ if (cmdline.hasOption("rps")) {
+ readersPerStream = Integer.parseInt(cmdline.getOptionValue("rps"));
+ }
+ if (cmdline.hasOption("msid")) {
+ maxStreamId = Integer.parseInt(cmdline.getOptionValue("msid"));
+ }
+ if (cmdline.hasOption("ti")) {
+ truncationInterval = Integer.parseInt(cmdline.getOptionValue("ti"));
+ }
+ if (cmdline.hasOption("ssid")) {
+ startStreamId = Integer.parseInt(cmdline.getOptionValue("ssid"));
+ }
+ if (cmdline.hasOption("esid")) {
+ endStreamId = Integer.parseInt(cmdline.getOptionValue("esid"));
+ }
+ if (cmdline.hasOption("hccs")) {
+ hostConnectionCoreSize = Integer.parseInt(cmdline.getOptionValue("hccs"));
+ }
+ if (cmdline.hasOption("hcl")) {
+ hostConnectionLimit = Integer.parseInt(cmdline.getOptionValue("hcl"));
+ }
+ if (cmdline.hasOption("sb")) {
+ sendBufferSize = Integer.parseInt(cmdline.getOptionValue("sb"));
+ }
+ if (cmdline.hasOption("rb")) {
+ recvBufferSize = Integer.parseInt(cmdline.getOptionValue("rb"));
+ }
+ if (cmdline.hasOption("rs")) {
+ routingServiceFinagleNameString = cmdline.getOptionValue("rs");
+ }
+ thriftmux = cmdline.hasOption("mx");
+ handshakeWithClientInfo = cmdline.hasOption("hsci");
+ readFromHead = cmdline.hasOption("rfh");
+ enableBatching = cmdline.hasOption("bt");
+ if (cmdline.hasOption("bbs")) {
+ batchBufferSize = Integer.parseInt(cmdline.getOptionValue("bbs"));
+ }
+ if (cmdline.hasOption("bfi")) {
+ batchFlushIntervalMicros = Integer.parseInt(cmdline.getOptionValue("bfi"));
+ }
+
+ checkArgument(shardId >= 0, "shardId must be >= 0");
+ checkArgument(numStreams > 0, "numStreams must be > 0");
+ checkArgument(durationMins > 0, "durationMins must be > 0");
+ checkArgument(streamPrefix != null, "streamPrefix must be defined");
+ checkArgument(hostConnectionCoreSize > 0, "host connection core size must be > 0");
+ checkArgument(hostConnectionLimit > 0, "host connection limit must be > 0");
+
+ if (cmdline.hasOption("p")) {
+ statsProvider = ReflectionUtils.newInstance(cmdline.getOptionValue("p"), StatsProvider.class);
+ } else {
+ statsProvider = new NullStatsProvider();
+ }
+
+ logger.info("Starting stats provider : {}.", statsProvider.getClass());
+ statsProvider.start(conf);
+
+ Worker w = null;
+ if (mode.startsWith("read")) {
+ w = runReader();
+ } else if (mode.startsWith("write")) {
+ w = runWriter();
+ } else if (mode.startsWith("dlwrite")) {
+ w = runDLWriter();
+ } else if (mode.startsWith("dlread")) {
+ w = runDLReader();
+ }
+
+ if (w == null) {
+ throw new IOException("Unknown mode " + mode + " to run the benchmark.");
+ }
+
+ Thread workerThread = new Thread(w, mode + "-benchmark-thread");
+ workerThread.start();
+
+ TimeUnit.MINUTES.sleep(durationMins);
+
+ logger.info("{} minutes passed, exiting...", durationMins);
+ w.close();
+
+ if (null != statsProvider) {
+ statsProvider.stop();
+ }
+
+ Runtime.getRuntime().exit(0);
+ }
+
+ Worker runWriter() {
+ checkArgument(!finagleNames.isEmpty() || !serversetPaths.isEmpty() || null != dlUri,
+ "either serverset paths, finagle-names or uri required");
+ checkArgument(msgSize > 0, "messagesize must be greater than 0");
+ checkArgument(rate > 0, "rate must be greater than 0");
+ checkArgument(maxRate >= rate, "max rate must be greater than rate");
+ checkArgument(changeRate >= 0, "change rate must be positive");
+ checkArgument(changeRateSeconds >= 0, "change rate must be positive");
+ checkArgument(concurrency > 0, "concurrency must be greater than 0");
+
+ ShiftableRateLimiter rateLimiter =
+ new ShiftableRateLimiter(rate, maxRate, changeRate, changeRateSeconds, TimeUnit.SECONDS);
+ return createWriteWorker(
+ streamPrefix,
+ dlUri,
+ null == startStreamId ? shardId * numStreams : startStreamId,
+ null == endStreamId ? (shardId + 1) * numStreams : endStreamId,
+ rateLimiter,
+ concurrency,
+ msgSize,
+ batchSize,
+ hostConnectionCoreSize,
+ hostConnectionLimit,
+ serversetPaths,
+ finagleNames,
+ statsReceiver.scope("write_client"),
+ statsProvider.getStatsLogger("write"),
+ thriftmux,
+ handshakeWithClientInfo,
+ sendBufferSize,
+ recvBufferSize,
+ enableBatching,
+ batchBufferSize,
+ batchFlushIntervalMicros,
+ routingServiceFinagleNameString);
+ }
+
+ protected WriterWorker createWriteWorker(
+ String streamPrefix,
+ URI uri,
+ int startStreamId,
+ int endStreamId,
+ ShiftableRateLimiter rateLimiter,
+ int writeConcurrency,
+ int messageSizeBytes,
+ int batchSize,
+ int hostConnectionCoreSize,
+ int hostConnectionLimit,
+ List<String> serverSetPaths,
+ List<String> finagleNames,
+ StatsReceiver statsReceiver,
+ StatsLogger statsLogger,
+ boolean thriftmux,
+ boolean handshakeWithClientInfo,
+ int sendBufferSize,
+ int recvBufferSize,
+ boolean enableBatching,
+ int batchBufferSize,
+ int batchFlushIntervalMicros,
+ String routingServiceFinagleNameString) {
+ return new WriterWorker(
+ streamPrefix,
+ uri,
+ startStreamId,
+ endStreamId,
+ rateLimiter,
+ writeConcurrency,
+ messageSizeBytes,
+ batchSize,
+ hostConnectionCoreSize,
+ hostConnectionLimit,
+ serverSetPaths,
+ finagleNames,
+ statsReceiver,
+ statsLogger,
+ thriftmux,
+ handshakeWithClientInfo,
+ sendBufferSize,
+ recvBufferSize,
+ enableBatching,
+ batchBufferSize,
+ batchFlushIntervalMicros,
+ routingServiceFinagleNameString);
+ }
+
+ Worker runDLWriter() throws IOException {
+ checkNotNull(dlUri, "dlUri must be defined");
+ checkArgument(rate > 0, "rate must be greater than 0");
+ checkArgument(maxRate >= rate, "max rate must be greater than rate");
+ checkArgument(changeRate >= 0, "change rate must be positive");
+ checkArgument(changeRateSeconds >= 0, "change rate must be positive");
+ checkArgument(concurrency > 0, "concurrency must be greater than 0");
+
+ ShiftableRateLimiter rateLimiter =
+ new ShiftableRateLimiter(rate, maxRate, changeRate, changeRateSeconds, TimeUnit.SECONDS);
+
+ return new DLWriterWorker(conf,
+ dlUri,
+ streamPrefix,
+ shardId * numStreams,
+ (shardId + 1) * numStreams,
+ rateLimiter,
+ concurrency,
+ msgSize,
+ statsProvider.getStatsLogger("dlwrite"));
+ }
+
+ Worker runReader() throws IOException {
+ checkArgument(!finagleNames.isEmpty() || !serversetPaths.isEmpty() || null != dlUri,
+ "either serverset paths, finagle-names or dlUri required");
+ checkArgument(concurrency > 0, "concurrency must be greater than 0");
+ checkArgument(truncationInterval > 0, "truncation interval should be greater than 0");
+ return runReaderInternal(serversetPaths, finagleNames, truncationInterval);
+ }
+
+ Worker runDLReader() throws IOException {
+ return runReaderInternal(new ArrayList<String>(), new ArrayList<String>(), 0);
+ }
+
+ private Worker runReaderInternal(List<String> serversetPaths,
+ List<String> finagleNames,
+ int truncationInterval) throws IOException {
+ checkNotNull(dlUri);
+
+ int ssid = null == startStreamId ? shardId * numStreams : startStreamId;
+ int esid = null == endStreamId ? (shardId + readersPerStream) * numStreams : endStreamId;
+ if (null != maxStreamId) {
+ esid = Math.min(esid, maxStreamId);
+ }
+
+ return createReaderWorker(
+ conf,
+ dlUri,
+ streamPrefix,
+ ssid,
+ esid,
+ concurrency,
+ serversetPaths,
+ finagleNames,
+ truncationInterval,
+ readFromHead,
+ statsReceiver,
+ statsProvider.getStatsLogger("dlreader"));
+ }
+
+ protected ReaderWorker createReaderWorker(
+ DistributedLogConfiguration conf,
+ URI uri,
+ String streamPrefix,
+ int startStreamId,
+ int endStreamId,
+ int readThreadPoolSize,
+ List<String> serverSetPaths,
+ List<String> finagleNames,
+ int truncationIntervalInSeconds,
+ boolean readFromHead, /* read from the earliest data of log */
+ StatsReceiver statsReceiver,
+ StatsLogger statsLogger) throws IOException {
+ return new ReaderWorker(
+ conf,
+ uri,
+ streamPrefix,
+ startStreamId,
+ endStreamId,
+ readThreadPoolSize,
+ serverSetPaths,
+ finagleNames,
+ truncationIntervalInSeconds,
+ readFromHead,
+ statsReceiver,
+ statsLogger);
+ }
+
+ public static void main(String[] args) {
+ Benchmarker benchmarker = new Benchmarker(args);
+ try {
+ benchmarker.run();
+ } catch (Exception e) {
+ logger.info("Benchmark quit due to : ", e);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java
new file mode 100644
index 0000000..a5e7a0a
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java
@@ -0,0 +1,245 @@
+/**
+ * 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 org.apache.distributedlog.benchmark;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.benchmark.utils.ShiftableRateLimiter;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.SchedulerUtils;
+import com.twitter.util.FutureEventListener;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The benchmark for core library writer.
+ */
+public class DLWriterWorker implements Worker {
+
+ private static final Logger LOG = LoggerFactory.getLogger(DLWriterWorker.class);
+
+ static final int BACKOFF_MS = 200;
+
+ final String streamPrefix;
+ final int startStreamId;
+ final int endStreamId;
+ final int writeConcurrency;
+ final int messageSizeBytes;
+ final ExecutorService executorService;
+ final ScheduledExecutorService rescueService;
+ final ShiftableRateLimiter rateLimiter;
+ final Random random;
+ final DistributedLogNamespace namespace;
+ final List<DistributedLogManager> dlms;
+ final List<AsyncLogWriter> streamWriters;
+ final int numStreams;
+
+ volatile boolean running = true;
+
+ final StatsLogger statsLogger;
+ final OpStatsLogger requestStat;
+
+ public DLWriterWorker(DistributedLogConfiguration conf,
+ URI uri,
+ String streamPrefix,
+ int startStreamId,
+ int endStreamId,
+ ShiftableRateLimiter rateLimiter,
+ int writeConcurrency,
+ int messageSizeBytes,
+ StatsLogger statsLogger) throws IOException {
+ checkArgument(startStreamId <= endStreamId);
+ this.streamPrefix = streamPrefix;
+ this.startStreamId = startStreamId;
+ this.endStreamId = endStreamId;
+ this.rateLimiter = rateLimiter;
+ this.writeConcurrency = writeConcurrency;
+ this.messageSizeBytes = messageSizeBytes;
+ this.statsLogger = statsLogger;
+ this.requestStat = this.statsLogger.getOpStatsLogger("requests");
+ this.executorService = Executors.newCachedThreadPool();
+ this.rescueService = Executors.newSingleThreadScheduledExecutor();
+ this.random = new Random(System.currentTimeMillis());
+
+ this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+ .conf(conf)
+ .uri(uri)
+ .statsLogger(statsLogger.scope("dl"))
+ .build();
+ this.numStreams = endStreamId - startStreamId;
+ dlms = new ArrayList<DistributedLogManager>(numStreams);
+ streamWriters = new ArrayList<AsyncLogWriter>(numStreams);
+ final ConcurrentMap<String, AsyncLogWriter> writers = new ConcurrentHashMap<String, AsyncLogWriter>();
+ final CountDownLatch latch = new CountDownLatch(this.numStreams);
+ for (int i = startStreamId; i < endStreamId; i++) {
+ final String streamName = String.format("%s_%d", streamPrefix, i);
+ final DistributedLogManager dlm = namespace.openLog(streamName);
+ executorService.submit(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+ if (null != writers.putIfAbsent(streamName, writer)) {
+ FutureUtils.result(writer.asyncClose());
+ }
+ latch.countDown();
+ } catch (IOException e) {
+ LOG.error("Failed to intialize writer for stream : {}", streamName, e);
+ }
+
+ }
+ });
+ dlms.add(dlm);
+ }
+ try {
+ latch.await();
+ } catch (InterruptedException e) {
+ throw new IOException("Interrupted on initializing writers for streams.", e);
+ }
+ for (int i = startStreamId; i < endStreamId; i++) {
+ final String streamName = String.format("%s_%d", streamPrefix, i);
+ AsyncLogWriter writer = writers.get(streamName);
+ if (null == writer) {
+ throw new IOException("Writer for " + streamName + " never initialized.");
+ }
+ streamWriters.add(writer);
+ }
+ LOG.info("Writing to {} streams.", numStreams);
+ }
+
+ void rescueWriter(int idx, AsyncLogWriter writer) {
+ if (streamWriters.get(idx) == writer) {
+ try {
+ FutureUtils.result(writer.asyncClose());
+ } catch (IOException e) {
+ LOG.error("Failed to close writer for stream {}.", idx);
+ }
+ AsyncLogWriter newWriter = null;
+ try {
+ newWriter = dlms.get(idx).startAsyncLogSegmentNonPartitioned();
+ } catch (IOException e) {
+ LOG.error("Failed to create new writer for stream {}, backoff for {} ms.",
+ idx, BACKOFF_MS);
+ scheduleRescue(idx, writer, BACKOFF_MS);
+ }
+ streamWriters.set(idx, newWriter);
+ } else {
+ LOG.warn("AsyncLogWriter for stream {} was already rescued.", idx);
+ }
+ }
+
+ void scheduleRescue(final int idx, final AsyncLogWriter writer, int delayMs) {
+ Runnable r = new Runnable() {
+ @Override
+ public void run() {
+ rescueWriter(idx, writer);
+ }
+ };
+ if (delayMs > 0) {
+ rescueService.schedule(r, delayMs, TimeUnit.MILLISECONDS);
+ } else {
+ rescueService.submit(r);
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.running = false;
+ SchedulerUtils.shutdownScheduler(this.executorService, 2, TimeUnit.MINUTES);
+ SchedulerUtils.shutdownScheduler(this.rescueService, 2, TimeUnit.MINUTES);
+ for (AsyncLogWriter writer : streamWriters) {
+ FutureUtils.result(writer.asyncClose());
+ }
+ for (DistributedLogManager dlm : dlms) {
+ dlm.close();
+ }
+ namespace.close();
+ }
+
+ @Override
+ public void run() {
+ LOG.info("Starting dlwriter (concurrency = {}, prefix = {}, numStreams = {})",
+ new Object[] { writeConcurrency, streamPrefix, numStreams });
+ for (int i = 0; i < writeConcurrency; i++) {
+ executorService.submit(new Writer(i));
+ }
+ }
+
+ class Writer implements Runnable {
+
+ final int idx;
+
+ Writer(int idx) {
+ this.idx = idx;
+ }
+
+ @Override
+ public void run() {
+ LOG.info("Started writer {}.", idx);
+ while (running) {
+ final int streamIdx = random.nextInt(numStreams);
+ final AsyncLogWriter writer = streamWriters.get(streamIdx);
+ rateLimiter.getLimiter().acquire();
+ final long requestMillis = System.currentTimeMillis();
+ final byte[] data;
+ try {
+ data = Utils.generateMessage(requestMillis, messageSizeBytes);
+ } catch (TException e) {
+ LOG.error("Error on generating message : ", e);
+ break;
+ }
+ writer.write(new LogRecord(requestMillis, data)).addEventListener(new FutureEventListener<DLSN>() {
+ @Override
+ public void onSuccess(DLSN value) {
+ requestStat.registerSuccessfulEvent(System.currentTimeMillis() - requestMillis);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ requestStat.registerFailedEvent(System.currentTimeMillis() - requestMillis);
+ LOG.error("Failed to publish, rescue it : ", cause);
+ scheduleRescue(streamIdx, writer, 0);
+ }
+ });
+ }
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java
new file mode 100644
index 0000000..11cba6f
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java
@@ -0,0 +1,468 @@
+/**
+ * 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 org.apache.distributedlog.benchmark;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.AsyncLogReader;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogRecordSet;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.benchmark.thrift.Message;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.SchedulerUtils;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration$;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The benchmark for core library reader.
+ */
+public class ReaderWorker implements Worker {
+
+ private static final Logger LOG = LoggerFactory.getLogger(ReaderWorker.class);
+
+ static final int BACKOFF_MS = 200;
+
+ final String streamPrefix;
+ final int startStreamId;
+ final int endStreamId;
+ final ScheduledExecutorService executorService;
+ final ExecutorService callbackExecutor;
+ final DistributedLogNamespace namespace;
+ final DistributedLogManager[] dlms;
+ final AsyncLogReader[] logReaders;
+ final StreamReader[] streamReaders;
+ final int numStreams;
+ final boolean readFromHead;
+
+ final int truncationIntervalInSeconds;
+ // DL Client Related Variables
+ final DLZkServerSet[] serverSets;
+ final List<String> finagleNames;
+ final DistributedLogClient dlc;
+
+ volatile boolean running = true;
+
+ final StatsReceiver statsReceiver;
+ final StatsLogger statsLogger;
+ final OpStatsLogger e2eStat;
+ final OpStatsLogger deliveryStat;
+ final OpStatsLogger negativeE2EStat;
+ final OpStatsLogger negativeDeliveryStat;
+ final OpStatsLogger truncationStat;
+ final Counter invalidRecordsCounter;
+ final Counter outOfOrderSequenceIdCounter;
+
+ class StreamReader implements FutureEventListener<List<LogRecordWithDLSN>>, Runnable, Gauge<Number> {
+
+ final int streamIdx;
+ final String streamName;
+ DLSN prevDLSN = null;
+ long prevSequenceId = Long.MIN_VALUE;
+ private static final String gaugeLabel = "sequence_id";
+
+ StreamReader(int idx, StatsLogger statsLogger) {
+ this.streamIdx = idx;
+ int streamId = startStreamId + streamIdx;
+ streamName = String.format("%s_%d", streamPrefix, streamId);
+ statsLogger.scope(streamName).registerGauge(gaugeLabel, this);
+ }
+
+ @Override
+ public void onSuccess(final List<LogRecordWithDLSN> records) {
+ for (final LogRecordWithDLSN record : records) {
+ if (record.isRecordSet()) {
+ try {
+ processRecordSet(record);
+ } catch (IOException e) {
+ onFailure(e);
+ }
+ } else {
+ processRecord(record);
+ }
+ }
+ readLoop();
+ }
+
+ public void processRecordSet(final LogRecordWithDLSN record) throws IOException {
+ LogRecordSet.Reader reader = LogRecordSet.of(record);
+ LogRecordWithDLSN nextRecord = reader.nextRecord();
+ while (null != nextRecord) {
+ processRecord(nextRecord);
+ nextRecord = reader.nextRecord();
+ }
+ }
+
+ public void processRecord(final LogRecordWithDLSN record) {
+ Message msg;
+ try {
+ msg = Utils.parseMessage(record.getPayload());
+ } catch (TException e) {
+ invalidRecordsCounter.inc();
+ LOG.warn("Failed to parse record {} for stream {} : size = {} , ",
+ new Object[] { record, streamIdx, record.getPayload().length, e });
+ return;
+ }
+ long curTimeMillis = System.currentTimeMillis();
+ long e2eLatency = curTimeMillis - msg.getPublishTime();
+ long deliveryLatency = curTimeMillis - record.getTransactionId();
+ if (e2eLatency >= 0) {
+ e2eStat.registerSuccessfulEvent(e2eLatency);
+ } else {
+ negativeE2EStat.registerSuccessfulEvent(-e2eLatency);
+ }
+ if (deliveryLatency >= 0) {
+ deliveryStat.registerSuccessfulEvent(deliveryLatency);
+ } else {
+ negativeDeliveryStat.registerSuccessfulEvent(-deliveryLatency);
+ }
+
+ prevDLSN = record.getDlsn();
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ scheduleReinitStream(streamIdx).map(new Function<Void, Void>() {
+ @Override
+ public Void apply(Void value) {
+ prevDLSN = null;
+ prevSequenceId = Long.MIN_VALUE;
+ readLoop();
+ return null;
+ }
+ });
+ }
+
+ void readLoop() {
+ if (!running) {
+ return;
+ }
+ logReaders[streamIdx].readBulk(10).addEventListener(this);
+ }
+
+ @Override
+ public void run() {
+ final DLSN dlsnToTruncate = prevDLSN;
+ if (null == dlsnToTruncate) {
+ return;
+ }
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ dlc.truncate(streamName, dlsnToTruncate).addEventListener(
+ new FutureEventListener<Boolean>() {
+ @Override
+ public void onSuccess(Boolean value) {
+ truncationStat.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS));
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ truncationStat.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS));
+ LOG.error("Failed to truncate stream {} to {} : ",
+ new Object[]{streamName, dlsnToTruncate, cause});
+ }
+ });
+ }
+
+ @Override
+ public Number getDefaultValue() {
+ return Long.MIN_VALUE;
+ }
+
+ @Override
+ public synchronized Number getSample() {
+ return prevSequenceId;
+ }
+
+ void unregisterGauge() {
+ statsLogger.scope(streamName).unregisterGauge(gaugeLabel, this);
+ }
+ }
+
+ public ReaderWorker(DistributedLogConfiguration conf,
+ URI uri,
+ String streamPrefix,
+ int startStreamId,
+ int endStreamId,
+ int readThreadPoolSize,
+ List<String> serverSetPaths,
+ List<String> finagleNames,
+ int truncationIntervalInSeconds,
+ boolean readFromHead, /* read from the earliest data of log */
+ StatsReceiver statsReceiver,
+ StatsLogger statsLogger) throws IOException {
+ checkArgument(startStreamId <= endStreamId);
+ this.streamPrefix = streamPrefix;
+ this.startStreamId = startStreamId;
+ this.endStreamId = endStreamId;
+ this.truncationIntervalInSeconds = truncationIntervalInSeconds;
+ this.readFromHead = readFromHead;
+ this.statsReceiver = statsReceiver;
+ this.statsLogger = statsLogger;
+ this.e2eStat = this.statsLogger.getOpStatsLogger("e2e");
+ this.negativeE2EStat = this.statsLogger.getOpStatsLogger("e2eNegative");
+ this.deliveryStat = this.statsLogger.getOpStatsLogger("delivery");
+ this.negativeDeliveryStat = this.statsLogger.getOpStatsLogger("deliveryNegative");
+ this.truncationStat = this.statsLogger.getOpStatsLogger("truncation");
+ this.invalidRecordsCounter = this.statsLogger.getCounter("invalid_records");
+ this.outOfOrderSequenceIdCounter = this.statsLogger.getCounter("out_of_order_seq_id");
+ this.executorService = Executors.newScheduledThreadPool(
+ readThreadPoolSize, new ThreadFactoryBuilder().setNameFormat("benchmark.reader-%d").build());
+ this.callbackExecutor = Executors.newFixedThreadPool(
+ Runtime.getRuntime().availableProcessors(),
+ new ThreadFactoryBuilder().setNameFormat("benchmark.reader-callback-%d").build());
+ this.finagleNames = finagleNames;
+ this.serverSets = createServerSets(serverSetPaths);
+
+ conf.setDeserializeRecordSetOnReads(false);
+
+ if (truncationIntervalInSeconds > 0 && (!finagleNames.isEmpty() || !serverSetPaths.isEmpty())) {
+ // Construct client for truncation
+ DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
+ .clientId(ClientId$.MODULE$.apply("dlog_loadtest_reader"))
+ .clientBuilder(ClientBuilder.get()
+ .hostConnectionLimit(10)
+ .hostConnectionCoresize(10)
+ .tcpConnectTimeout(Duration$.MODULE$.fromSeconds(1))
+ .requestTimeout(Duration$.MODULE$.fromSeconds(2)))
+ .redirectBackoffStartMs(100)
+ .redirectBackoffMaxMs(500)
+ .requestTimeoutMs(2000)
+ .statsReceiver(statsReceiver)
+ .thriftmux(true)
+ .name("reader");
+
+ if (serverSetPaths.isEmpty()) {
+ // Prepare finagle names
+ String local = finagleNames.get(0);
+ String[] remotes = new String[finagleNames.size() - 1];
+ finagleNames.subList(1, finagleNames.size()).toArray(remotes);
+
+ builder = builder.finagleNameStrs(local, remotes);
+ LOG.info("Initialized distributedlog client for truncation @ {}.", finagleNames);
+ } else if (serverSets.length != 0){
+ ServerSet local = this.serverSets[0].getServerSet();
+ ServerSet[] remotes = new ServerSet[this.serverSets.length - 1];
+ for (int i = 1; i < serverSets.length; i++) {
+ remotes[i - 1] = serverSets[i].getServerSet();
+ }
+
+ builder = builder.serverSets(local, remotes);
+ LOG.info("Initialized distributedlog client for truncation @ {}.", serverSetPaths);
+ } else {
+ builder = builder.uri(uri);
+ LOG.info("Initialized distributedlog client for namespace {}", uri);
+ }
+ dlc = builder.build();
+ } else {
+ dlc = null;
+ }
+
+ // construct the factory
+ this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+ .conf(conf)
+ .uri(uri)
+ .statsLogger(statsLogger.scope("dl"))
+ .build();
+ this.numStreams = endStreamId - startStreamId;
+ this.dlms = new DistributedLogManager[numStreams];
+ this.logReaders = new AsyncLogReader[numStreams];
+ final CountDownLatch latch = new CountDownLatch(numStreams);
+ for (int i = 0; i < numStreams; i++) {
+ final int idx = i;
+ executorService.submit(new Runnable() {
+ @Override
+ public void run() {
+ reinitStream(idx).map(new Function<Void, Void>() {
+ @Override
+ public Void apply(Void value) {
+ LOG.info("Initialized stream reader {}.", idx);
+ latch.countDown();
+ return null;
+ }
+ });
+ }
+ });
+ }
+ try {
+ latch.await();
+ } catch (InterruptedException e) {
+ throw new DLInterruptedException("Failed to intialize benchmark readers : ", e);
+ }
+ this.streamReaders = new StreamReader[numStreams];
+ for (int i = 0; i < numStreams; i++) {
+ streamReaders[i] = new StreamReader(i, statsLogger.scope("perstream"));
+ if (truncationIntervalInSeconds > 0) {
+ executorService.scheduleWithFixedDelay(streamReaders[i],
+ truncationIntervalInSeconds, truncationIntervalInSeconds, TimeUnit.SECONDS);
+ }
+ }
+ LOG.info("Initialized benchmark reader on {} streams {} : [{} - {})",
+ new Object[] { numStreams, streamPrefix, startStreamId, endStreamId });
+ }
+
+ protected DLZkServerSet[] createServerSets(List<String> serverSetPaths) {
+ DLZkServerSet[] serverSets = new DLZkServerSet[serverSetPaths.size()];
+ for (int i = 0; i < serverSets.length; i++) {
+ String serverSetPath = serverSetPaths.get(i);
+ serverSets[i] = DLZkServerSet.of(URI.create(serverSetPath), 60000);
+ }
+ return serverSets;
+ }
+
+ private Future<Void> reinitStream(int idx) {
+ Promise<Void> promise = new Promise<Void>();
+ reinitStream(idx, promise);
+ return promise;
+ }
+
+ private void reinitStream(int idx, Promise<Void> promise) {
+ int streamId = startStreamId + idx;
+ String streamName = String.format("%s_%d", streamPrefix, streamId);
+
+ if (logReaders[idx] != null) {
+ try {
+ FutureUtils.result(logReaders[idx].asyncClose());
+ } catch (IOException e) {
+ LOG.warn("Failed on closing stream reader {} : ", streamName, e);
+ }
+ logReaders[idx] = null;
+ }
+ if (dlms[idx] != null) {
+ try {
+ dlms[idx].close();
+ } catch (IOException e) {
+ LOG.warn("Failed on closing dlm {} : ", streamName, e);
+ }
+ dlms[idx] = null;
+ }
+
+ try {
+ dlms[idx] = namespace.openLog(streamName);
+ } catch (IOException ioe) {
+ LOG.error("Failed on creating dlm {} : ", streamName, ioe);
+ scheduleReinitStream(idx, promise);
+ return;
+ }
+ DLSN lastDLSN;
+ if (readFromHead) {
+ lastDLSN = DLSN.InitialDLSN;
+ } else {
+ try {
+ lastDLSN = dlms[idx].getLastDLSN();
+ } catch (IOException ioe) {
+ LOG.error("Failed on getting last dlsn from stream {} : ", streamName, ioe);
+ scheduleReinitStream(idx, promise);
+ return;
+ }
+ }
+ try {
+ logReaders[idx] = dlms[idx].getAsyncLogReader(lastDLSN);
+ } catch (IOException ioe) {
+ LOG.error("Failed on opening reader for stream {} starting from {} : ",
+ new Object[] { streamName, lastDLSN, ioe });
+ scheduleReinitStream(idx, promise);
+ return;
+ }
+ LOG.info("Opened reader for stream {}, starting from {}.", streamName, lastDLSN);
+ promise.setValue(null);
+ }
+
+ Future<Void> scheduleReinitStream(int idx) {
+ Promise<Void> promise = new Promise<Void>();
+ scheduleReinitStream(idx, promise);
+ return promise;
+ }
+
+ void scheduleReinitStream(final int idx, final Promise<Void> promise) {
+ executorService.schedule(new Runnable() {
+ @Override
+ public void run() {
+ reinitStream(idx, promise);
+ }
+ }, BACKOFF_MS, TimeUnit.MILLISECONDS);
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.running = false;
+ for (AsyncLogReader reader : logReaders) {
+ if (null != reader) {
+ FutureUtils.result(reader.asyncClose());
+ }
+ }
+ for (DistributedLogManager dlm : dlms) {
+ if (null != dlm) {
+ dlm.close();
+ }
+ }
+ namespace.close();
+ SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
+ SchedulerUtils.shutdownScheduler(callbackExecutor, 2, TimeUnit.MINUTES);
+ if (this.dlc != null) {
+ this.dlc.close();
+ }
+ for (DLZkServerSet serverSet: serverSets) {
+ serverSet.close();
+ }
+ // Unregister gauges to prevent GC spirals
+ for (StreamReader sr : streamReaders) {
+ sr.unregisterGauge();
+ }
+ }
+
+ @Override
+ public void run() {
+ LOG.info("Starting reader (prefix = {}, numStreams = {}).",
+ streamPrefix, numStreams);
+ for (StreamReader sr : streamReaders) {
+ sr.readLoop();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Utils.java
----------------------------------------------------------------------
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Utils.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Utils.java
new file mode 100644
index 0000000..81f99ef
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Utils.java
@@ -0,0 +1,57 @@
+/**
+ * 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 org.apache.distributedlog.benchmark;
+
+import org.apache.distributedlog.benchmark.thrift.Message;
+import java.nio.ByteBuffer;
+import java.util.Random;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.transport.TMemoryInputTransport;
+
+/**
+ * Utils for generating and parsing messages.
+ */
+public class Utils {
+
+ static final Random RAND = new Random(System.currentTimeMillis());
+ static final ThreadLocal<TSerializer> MSG_SERIALIZER =
+ new ThreadLocal<TSerializer>() {
+ @Override
+ public TSerializer initialValue() {
+ return new TSerializer(new TBinaryProtocol.Factory());
+ }
+ };
+
+ public static byte[] generateMessage(long requestMillis, int payLoadSize) throws TException {
+ byte[] payload = new byte[payLoadSize];
+ RAND.nextBytes(payload);
+ Message msg = new Message(requestMillis, ByteBuffer.wrap(payload));
+ return MSG_SERIALIZER.get().serialize(msg);
+ }
+
+ public static Message parseMessage(byte[] data) throws TException {
+ Message msg = new Message();
+ TMemoryInputTransport transport = new TMemoryInputTransport(data);
+ TBinaryProtocol protocol = new TBinaryProtocol(transport);
+ msg.read(protocol);
+ return msg;
+ }
+
+}
[10/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
new file mode 100644
index 0000000..07ae0ff
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
@@ -0,0 +1,715 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.exceptions.LogEmptyException;
+import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.metadata.LogMetadata;
+import org.apache.distributedlog.io.AsyncAbortable;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
+import org.apache.distributedlog.logsegment.PerStreamLogSegmentCache;
+import org.apache.distributedlog.logsegment.LogSegmentFilter;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
+import org.apache.distributedlog.metadata.LogStreamMetadataStore;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * The base class about log handler on managing log segments.
+ *
+ * <h3>Metrics</h3>
+ * The log handler is a base class on managing log segments. so all the metrics
+ * here are related to log segments retrieval and exposed under `logsegments`.
+ * These metrics are all OpStats, in the format of <code>`scope`/logsegments/`op`</code>.
+ * <p>
+ * Those operations are:
+ * <ul>
+ * <li>get_inprogress_segment: time between the inprogress log segment created and
+ * the handler read it.
+ * <li>get_completed_segment: time between a log segment is turned to completed and
+ * the handler read it.
+ * <li>negative_get_inprogress_segment: record the negative values for `get_inprogress_segment`.
+ * <li>negative_get_completed_segment: record the negative values for `get_completed_segment`.
+ * <li>recover_last_entry: recovering last entry from a log segment
+ * <li>recover_scanned_entries: the number of entries that are scanned during recovering.
+ * </ul>
+ * @see BKLogWriteHandler
+ * @see BKLogReadHandler
+ */
+public abstract class BKLogHandler implements AsyncCloseable, AsyncAbortable {
+ static final Logger LOG = LoggerFactory.getLogger(BKLogHandler.class);
+
+ protected final LogMetadata logMetadata;
+ protected final DistributedLogConfiguration conf;
+ protected final LogStreamMetadataStore streamMetadataStore;
+ protected final LogSegmentMetadataStore metadataStore;
+ protected final LogSegmentMetadataCache metadataCache;
+ protected final LogSegmentEntryStore entryStore;
+ protected final int firstNumEntriesPerReadLastRecordScan;
+ protected final int maxNumEntriesPerReadLastRecordScan;
+ protected volatile long lastLedgerRollingTimeMillis = -1;
+ protected final OrderedScheduler scheduler;
+ protected final StatsLogger statsLogger;
+ protected final AlertStatsLogger alertStatsLogger;
+ protected volatile boolean reportGetSegmentStats = false;
+ private final String lockClientId;
+ protected final AtomicReference<IOException> metadataException = new AtomicReference<IOException>(null);
+
+ // Maintain the list of log segments per stream
+ protected final PerStreamLogSegmentCache logSegmentCache;
+
+ // trace
+ protected final long metadataLatencyWarnThresholdMillis;
+
+ // Stats
+ private final OpStatsLogger getInprogressSegmentStat;
+ private final OpStatsLogger getCompletedSegmentStat;
+ private final OpStatsLogger negativeGetInprogressSegmentStat;
+ private final OpStatsLogger negativeGetCompletedSegmentStat;
+ private final OpStatsLogger recoverLastEntryStats;
+ private final OpStatsLogger recoverScannedEntriesStats;
+
+ /**
+ * Construct a Bookkeeper journal manager.
+ */
+ BKLogHandler(LogMetadata metadata,
+ DistributedLogConfiguration conf,
+ LogStreamMetadataStore streamMetadataStore,
+ LogSegmentMetadataCache metadataCache,
+ LogSegmentEntryStore entryStore,
+ OrderedScheduler scheduler,
+ StatsLogger statsLogger,
+ AlertStatsLogger alertStatsLogger,
+ String lockClientId) {
+ this.logMetadata = metadata;
+ this.conf = conf;
+ this.scheduler = scheduler;
+ this.statsLogger = statsLogger;
+ this.alertStatsLogger = alertStatsLogger;
+ this.logSegmentCache = new PerStreamLogSegmentCache(
+ metadata.getLogName(),
+ conf.isLogSegmentSequenceNumberValidationEnabled());
+ firstNumEntriesPerReadLastRecordScan = conf.getFirstNumEntriesPerReadLastRecordScan();
+ maxNumEntriesPerReadLastRecordScan = conf.getMaxNumEntriesPerReadLastRecordScan();
+ this.streamMetadataStore = streamMetadataStore;
+ this.metadataStore = streamMetadataStore.getLogSegmentMetadataStore();
+ this.metadataCache = metadataCache;
+ this.entryStore = entryStore;
+ this.lockClientId = lockClientId;
+
+ // Traces
+ this.metadataLatencyWarnThresholdMillis = conf.getMetadataLatencyWarnThresholdMillis();
+
+ // Stats
+ StatsLogger segmentsLogger = statsLogger.scope("logsegments");
+ getInprogressSegmentStat = segmentsLogger.getOpStatsLogger("get_inprogress_segment");
+ getCompletedSegmentStat = segmentsLogger.getOpStatsLogger("get_completed_segment");
+ negativeGetInprogressSegmentStat = segmentsLogger.getOpStatsLogger("negative_get_inprogress_segment");
+ negativeGetCompletedSegmentStat = segmentsLogger.getOpStatsLogger("negative_get_completed_segment");
+ recoverLastEntryStats = segmentsLogger.getOpStatsLogger("recover_last_entry");
+ recoverScannedEntriesStats = segmentsLogger.getOpStatsLogger("recover_scanned_entries");
+ }
+
+ BKLogHandler checkMetadataException() throws IOException {
+ if (null != metadataException.get()) {
+ throw metadataException.get();
+ }
+ return this;
+ }
+
+ public void reportGetSegmentStats(boolean enabled) {
+ this.reportGetSegmentStats = enabled;
+ }
+
+ public String getLockClientId() {
+ return lockClientId;
+ }
+
+ public Future<LogRecordWithDLSN> asyncGetFirstLogRecord() {
+ final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
+ streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
+ .addEventListener(new FutureEventListener<Void>() {
+ @Override
+ public void onSuccess(Void value) {
+ readLogSegmentsFromStore(
+ LogSegmentMetadata.COMPARATOR,
+ LogSegmentFilter.DEFAULT_FILTER,
+ null
+ ).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+
+ @Override
+ public void onSuccess(Versioned<List<LogSegmentMetadata>> ledgerList) {
+ if (ledgerList.getValue().isEmpty()) {
+ promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+ return;
+ }
+ Future<LogRecordWithDLSN> firstRecord = null;
+ for (LogSegmentMetadata ledger : ledgerList.getValue()) {
+ if (!ledger.isTruncated() && (ledger.getRecordCount() > 0 || ledger.isInProgress())) {
+ firstRecord = asyncReadFirstUserRecord(ledger, DLSN.InitialDLSN);
+ break;
+ }
+ }
+ if (null != firstRecord) {
+ promise.become(firstRecord);
+ } else {
+ promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ promise.setException(cause);
+ }
+ });
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ promise.setException(cause);
+ }
+ });
+ return promise;
+ }
+
+ public Future<LogRecordWithDLSN> getLastLogRecordAsync(final boolean recover, final boolean includeEndOfStream) {
+ final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
+ streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
+ .addEventListener(new FutureEventListener<Void>() {
+ @Override
+ public void onSuccess(Void value) {
+ readLogSegmentsFromStore(
+ LogSegmentMetadata.DESC_COMPARATOR,
+ LogSegmentFilter.DEFAULT_FILTER,
+ null
+ ).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+
+ @Override
+ public void onSuccess(Versioned<List<LogSegmentMetadata>> ledgerList) {
+ if (ledgerList.getValue().isEmpty()) {
+ promise.setException(
+ new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+ return;
+ }
+ asyncGetLastLogRecord(
+ ledgerList.getValue().iterator(),
+ promise,
+ recover,
+ false,
+ includeEndOfStream);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ promise.setException(cause);
+ }
+ });
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ promise.setException(cause);
+ }
+ });
+ return promise;
+ }
+
+ private void asyncGetLastLogRecord(final Iterator<LogSegmentMetadata> ledgerIter,
+ final Promise<LogRecordWithDLSN> promise,
+ final boolean fence,
+ final boolean includeControlRecord,
+ final boolean includeEndOfStream) {
+ if (ledgerIter.hasNext()) {
+ LogSegmentMetadata metadata = ledgerIter.next();
+ asyncReadLastRecord(metadata, fence, includeControlRecord, includeEndOfStream).addEventListener(
+ new FutureEventListener<LogRecordWithDLSN>() {
+ @Override
+ public void onSuccess(LogRecordWithDLSN record) {
+ if (null == record) {
+ asyncGetLastLogRecord(ledgerIter, promise, fence, includeControlRecord, includeEndOfStream);
+ } else {
+ promise.setValue(record);
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ promise.setException(cause);
+ }
+ }
+ );
+ } else {
+ promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+ }
+ }
+
+ private Future<LogRecordWithDLSN> asyncReadFirstUserRecord(LogSegmentMetadata ledger, DLSN beginDLSN) {
+ return ReadUtils.asyncReadFirstUserRecord(
+ getFullyQualifiedName(),
+ ledger,
+ firstNumEntriesPerReadLastRecordScan,
+ maxNumEntriesPerReadLastRecordScan,
+ new AtomicInteger(0),
+ scheduler,
+ entryStore,
+ beginDLSN
+ );
+ }
+
+ /**
+ * This is a helper method to compactly return the record count between two records, the first denoted by
+ * beginDLSN and the second denoted by endPosition. Its up to the caller to ensure that endPosition refers to
+ * position in the same ledger as beginDLSN.
+ */
+ private Future<Long> asyncGetLogRecordCount(LogSegmentMetadata ledger, final DLSN beginDLSN, final long endPosition) {
+ return asyncReadFirstUserRecord(ledger, beginDLSN).map(new Function<LogRecordWithDLSN, Long>() {
+ public Long apply(final LogRecordWithDLSN beginRecord) {
+ long recordCount = 0;
+ if (null != beginRecord) {
+ recordCount = endPosition + 1 - beginRecord.getLastPositionWithinLogSegment();
+ }
+ return recordCount;
+ }
+ });
+ }
+
+ /**
+ * Ledger metadata tells us how many records are in each completed segment, but for the first and last segments
+ * we may have to crack open the entry and count. For the first entry, we need to do so because beginDLSN may be
+ * an interior entry. For the last entry, if it is inprogress, we need to recover it and find the last user
+ * entry.
+ */
+ private Future<Long> asyncGetLogRecordCount(final LogSegmentMetadata ledger, final DLSN beginDLSN) {
+ if (ledger.isInProgress() && ledger.isDLSNinThisSegment(beginDLSN)) {
+ return asyncReadLastUserRecord(ledger).flatMap(new Function<LogRecordWithDLSN, Future<Long>>() {
+ public Future<Long> apply(final LogRecordWithDLSN endRecord) {
+ if (null != endRecord) {
+ return asyncGetLogRecordCount(ledger, beginDLSN, endRecord.getLastPositionWithinLogSegment() /* end position */);
+ } else {
+ return Future.value((long) 0);
+ }
+ }
+ });
+ } else if (ledger.isInProgress()) {
+ return asyncReadLastUserRecord(ledger).map(new Function<LogRecordWithDLSN, Long>() {
+ public Long apply(final LogRecordWithDLSN endRecord) {
+ if (null != endRecord) {
+ return (long) endRecord.getLastPositionWithinLogSegment();
+ } else {
+ return (long) 0;
+ }
+ }
+ });
+ } else if (ledger.isDLSNinThisSegment(beginDLSN)) {
+ return asyncGetLogRecordCount(ledger, beginDLSN, ledger.getRecordCount() /* end position */);
+ } else {
+ return Future.value((long) ledger.getRecordCount());
+ }
+ }
+
+ /**
+ * Get a count of records between beginDLSN and the end of the stream.
+ *
+ * @param beginDLSN dlsn marking the start of the range
+ * @return the count of records present in the range
+ */
+ public Future<Long> asyncGetLogRecordCount(final DLSN beginDLSN) {
+ return streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
+ .flatMap(new Function<Void, Future<Long>>() {
+ public Future<Long> apply(Void done) {
+
+ return readLogSegmentsFromStore(
+ LogSegmentMetadata.COMPARATOR,
+ LogSegmentFilter.DEFAULT_FILTER,
+ null
+ ).flatMap(new Function<Versioned<List<LogSegmentMetadata>>, Future<Long>>() {
+ public Future<Long> apply(Versioned<List<LogSegmentMetadata>> ledgerList) {
+
+ List<Future<Long>> futureCounts = new ArrayList<Future<Long>>(ledgerList.getValue().size());
+ for (LogSegmentMetadata ledger : ledgerList.getValue()) {
+ if (ledger.getLogSegmentSequenceNumber() >= beginDLSN.getLogSegmentSequenceNo()) {
+ futureCounts.add(asyncGetLogRecordCount(ledger, beginDLSN));
+ }
+ }
+ return Future.collect(futureCounts).map(new Function<List<Long>, Long>() {
+ public Long apply(List<Long> counts) {
+ return sum(counts);
+ }
+ });
+ }
+ });
+ }
+ });
+ }
+
+ private Long sum(List<Long> values) {
+ long sum = 0;
+ for (Long value : values) {
+ sum += value;
+ }
+ return sum;
+ }
+
+ @Override
+ public Future<Void> asyncAbort() {
+ return asyncClose();
+ }
+
+ public Future<LogRecordWithDLSN> asyncReadLastUserRecord(final LogSegmentMetadata l) {
+ return asyncReadLastRecord(l, false, false, false);
+ }
+
+ public Future<LogRecordWithDLSN> asyncReadLastRecord(final LogSegmentMetadata l,
+ final boolean fence,
+ final boolean includeControl,
+ final boolean includeEndOfStream) {
+ final AtomicInteger numRecordsScanned = new AtomicInteger(0);
+ final Stopwatch stopwatch = Stopwatch.createStarted();
+ return ReadUtils.asyncReadLastRecord(
+ getFullyQualifiedName(),
+ l,
+ fence,
+ includeControl,
+ includeEndOfStream,
+ firstNumEntriesPerReadLastRecordScan,
+ maxNumEntriesPerReadLastRecordScan,
+ numRecordsScanned,
+ scheduler,
+ entryStore
+ ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+ @Override
+ public void onSuccess(LogRecordWithDLSN value) {
+ recoverLastEntryStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ recoverScannedEntriesStats.registerSuccessfulEvent(numRecordsScanned.get());
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ recoverLastEntryStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+ }
+ });
+ }
+
+ protected void setLastLedgerRollingTimeMillis(long rollingTimeMillis) {
+ if (lastLedgerRollingTimeMillis < rollingTimeMillis) {
+ lastLedgerRollingTimeMillis = rollingTimeMillis;
+ }
+ }
+
+ public String getFullyQualifiedName() {
+ return logMetadata.getFullyQualifiedName();
+ }
+
+ // Log Segments Related Functions
+ //
+ // ***Note***
+ // Get log segment list should go through #getCachedLogSegments as we need to assign start sequence id
+ // for inprogress log segment so the reader could generate the right sequence id.
+ //
+ // ***PerStreamCache vs LogSegmentMetadataCache **
+ // The per stream cache maintains the list of segments per stream, while the metadata cache
+ // maintains log segments. The metadata cache is just to reduce the access to zookeeper, it is
+ // okay that some of the log segments are not in the cache; however the per stream cache can not
+ // have any gaps between log segment sequence numbers which it has to be accurate.
+
+ /**
+ * Get the cached log segments.
+ *
+ * @param comparator the comparator to sort the returned log segments.
+ * @return list of sorted log segments
+ * @throws UnexpectedException if unexpected condition detected.
+ */
+ protected List<LogSegmentMetadata> getCachedLogSegments(Comparator<LogSegmentMetadata> comparator)
+ throws UnexpectedException {
+ try {
+ return logSegmentCache.getLogSegments(comparator);
+ } catch (UnexpectedException ue) {
+ // the log segments cache went wrong
+ LOG.error("Unexpected exception on getting log segments from the cache for stream {}",
+ getFullyQualifiedName(), ue);
+ metadataException.compareAndSet(null, ue);
+ throw ue;
+ }
+ }
+
+ /**
+ * Add the segment <i>metadata</i> for <i>name</i> in the cache.
+ *
+ * @param name
+ * segment znode name.
+ * @param metadata
+ * segment metadata.
+ */
+ protected void addLogSegmentToCache(String name, LogSegmentMetadata metadata) {
+ metadataCache.put(metadata.getZkPath(), metadata);
+ logSegmentCache.add(name, metadata);
+ // update the last ledger rolling time
+ if (!metadata.isInProgress() && (lastLedgerRollingTimeMillis < metadata.getCompletionTime())) {
+ lastLedgerRollingTimeMillis = metadata.getCompletionTime();
+ }
+
+ if (reportGetSegmentStats) {
+ // update stats
+ long ts = System.currentTimeMillis();
+ if (metadata.isInProgress()) {
+ // as we used timestamp as start tx id we could take it as start time
+ // NOTE: it is a hack here.
+ long elapsedMillis = ts - metadata.getFirstTxId();
+ long elapsedMicroSec = TimeUnit.MILLISECONDS.toMicros(elapsedMillis);
+ if (elapsedMicroSec > 0) {
+ if (elapsedMillis > metadataLatencyWarnThresholdMillis) {
+ LOG.warn("{} received inprogress log segment in {} millis: {}",
+ new Object[] { getFullyQualifiedName(), elapsedMillis, metadata });
+ }
+ getInprogressSegmentStat.registerSuccessfulEvent(elapsedMicroSec);
+ } else {
+ negativeGetInprogressSegmentStat.registerSuccessfulEvent(-elapsedMicroSec);
+ }
+ } else {
+ long elapsedMillis = ts - metadata.getCompletionTime();
+ long elapsedMicroSec = TimeUnit.MILLISECONDS.toMicros(elapsedMillis);
+ if (elapsedMicroSec > 0) {
+ if (elapsedMillis > metadataLatencyWarnThresholdMillis) {
+ LOG.warn("{} received completed log segment in {} millis : {}",
+ new Object[] { getFullyQualifiedName(), elapsedMillis, metadata });
+ }
+ getCompletedSegmentStat.registerSuccessfulEvent(elapsedMicroSec);
+ } else {
+ negativeGetCompletedSegmentStat.registerSuccessfulEvent(-elapsedMicroSec);
+ }
+ }
+ }
+ }
+
+ /**
+ * Read log segment <i>name</i> from the cache.
+ *
+ * @param name name of the log segment
+ * @return log segment metadata
+ */
+ protected LogSegmentMetadata readLogSegmentFromCache(String name) {
+ return logSegmentCache.get(name);
+ }
+
+ /**
+ * Remove the log segment <i>name</i> from the cache.
+ *
+ * @param name name of the log segment.
+ * @return log segment metadata
+ */
+ protected LogSegmentMetadata removeLogSegmentFromCache(String name) {
+ metadataCache.invalidate(name);
+ return logSegmentCache.remove(name);
+ }
+
+ /**
+ * Update the log segment cache with updated mapping
+ *
+ * @param logSegmentsRemoved log segments removed
+ * @param logSegmentsAdded log segments added
+ */
+ protected void updateLogSegmentCache(Set<String> logSegmentsRemoved,
+ Map<String, LogSegmentMetadata> logSegmentsAdded) {
+ for (String segmentName : logSegmentsRemoved) {
+ metadataCache.invalidate(segmentName);
+ }
+ for (Map.Entry<String, LogSegmentMetadata> entry : logSegmentsAdded.entrySet()) {
+ metadataCache.put(entry.getKey(), entry.getValue());
+ }
+ logSegmentCache.update(logSegmentsRemoved, logSegmentsAdded);
+ }
+
+ /**
+ * Read the log segments from the store and register a listener
+ * @param comparator
+ * @param segmentFilter
+ * @param logSegmentNamesListener
+ * @return future represents the result of log segments
+ */
+ public Future<Versioned<List<LogSegmentMetadata>>> readLogSegmentsFromStore(
+ final Comparator<LogSegmentMetadata> comparator,
+ final LogSegmentFilter segmentFilter,
+ final LogSegmentNamesListener logSegmentNamesListener) {
+ final Promise<Versioned<List<LogSegmentMetadata>>> readResult =
+ new Promise<Versioned<List<LogSegmentMetadata>>>();
+ metadataStore.getLogSegmentNames(logMetadata.getLogSegmentsPath(), logSegmentNamesListener)
+ .addEventListener(new FutureEventListener<Versioned<List<String>>>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ FutureUtils.setException(readResult, cause);
+ }
+
+ @Override
+ public void onSuccess(Versioned<List<String>> logSegmentNames) {
+ readLogSegmentsFromStore(logSegmentNames, comparator, segmentFilter, readResult);
+ }
+ });
+ return readResult;
+ }
+
+ protected void readLogSegmentsFromStore(final Versioned<List<String>> logSegmentNames,
+ final Comparator<LogSegmentMetadata> comparator,
+ final LogSegmentFilter segmentFilter,
+ final Promise<Versioned<List<LogSegmentMetadata>>> readResult) {
+ Set<String> segmentsReceived = new HashSet<String>();
+ segmentsReceived.addAll(segmentFilter.filter(logSegmentNames.getValue()));
+ Set<String> segmentsAdded;
+ final Set<String> removedSegments = Collections.synchronizedSet(new HashSet<String>());
+ final Map<String, LogSegmentMetadata> addedSegments =
+ Collections.synchronizedMap(new HashMap<String, LogSegmentMetadata>());
+ Pair<Set<String>, Set<String>> segmentChanges = logSegmentCache.diff(segmentsReceived);
+ segmentsAdded = segmentChanges.getLeft();
+ removedSegments.addAll(segmentChanges.getRight());
+
+ if (segmentsAdded.isEmpty()) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("No segments added for {}.", getFullyQualifiedName());
+ }
+
+ // update the cache before #getCachedLogSegments to return
+ updateLogSegmentCache(removedSegments, addedSegments);
+
+ List<LogSegmentMetadata> segmentList;
+ try {
+ segmentList = getCachedLogSegments(comparator);
+ } catch (UnexpectedException e) {
+ FutureUtils.setException(readResult, e);
+ return;
+ }
+
+ FutureUtils.setValue(readResult,
+ new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNames.getVersion()));
+ return;
+ }
+
+ final AtomicInteger numChildren = new AtomicInteger(segmentsAdded.size());
+ final AtomicInteger numFailures = new AtomicInteger(0);
+ for (final String segment: segmentsAdded) {
+ String logSegmentPath = logMetadata.getLogSegmentPath(segment);
+ LogSegmentMetadata cachedSegment = metadataCache.get(logSegmentPath);
+ if (null != cachedSegment) {
+ addedSegments.put(segment, cachedSegment);
+ completeReadLogSegmentsFromStore(
+ removedSegments,
+ addedSegments,
+ comparator,
+ readResult,
+ logSegmentNames.getVersion(),
+ numChildren,
+ numFailures);
+ continue;
+ }
+ metadataStore.getLogSegment(logSegmentPath)
+ .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+
+ @Override
+ public void onSuccess(LogSegmentMetadata result) {
+ addedSegments.put(segment, result);
+ complete();
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ // LogSegmentNotFoundException exception is possible in two cases
+ // 1. A log segment was deleted by truncation between the call to getChildren and read
+ // attempt on the znode corresponding to the segment
+ // 2. In progress segment has been completed => inprogress ZNode does not exist
+ if (cause instanceof LogSegmentNotFoundException) {
+ removedSegments.add(segment);
+ complete();
+ } else {
+ // fail fast
+ if (1 == numFailures.incrementAndGet()) {
+ FutureUtils.setException(readResult, cause);
+ return;
+ }
+ }
+ }
+
+ private void complete() {
+ completeReadLogSegmentsFromStore(
+ removedSegments,
+ addedSegments,
+ comparator,
+ readResult,
+ logSegmentNames.getVersion(),
+ numChildren,
+ numFailures);
+ }
+ });
+ }
+ }
+
+ private void completeReadLogSegmentsFromStore(final Set<String> removedSegments,
+ final Map<String, LogSegmentMetadata> addedSegments,
+ final Comparator<LogSegmentMetadata> comparator,
+ final Promise<Versioned<List<LogSegmentMetadata>>> readResult,
+ final Version logSegmentNamesVersion,
+ final AtomicInteger numChildren,
+ final AtomicInteger numFailures) {
+ if (0 != numChildren.decrementAndGet()) {
+ return;
+ }
+ if (numFailures.get() > 0) {
+ return;
+ }
+ // update the cache only when fetch completed and before #getCachedLogSegments
+ updateLogSegmentCache(removedSegments, addedSegments);
+ List<LogSegmentMetadata> segmentList;
+ try {
+ segmentList = getCachedLogSegments(comparator);
+ } catch (UnexpectedException e) {
+ FutureUtils.setException(readResult, e);
+ return;
+ }
+ FutureUtils.setValue(readResult,
+ new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNamesVersion));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
new file mode 100644
index 0000000..c6e2e07
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
@@ -0,0 +1,431 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.metadata.LogMetadataForReader;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentFilter;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
+import org.apache.distributedlog.metadata.LogStreamMetadataStore;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.Utils;
+import com.twitter.util.ExceptionalFunction;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import com.twitter.util.Return;
+import com.twitter.util.Throw;
+import com.twitter.util.Try;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * Log Handler for Readers.
+ * <h3>Metrics</h3>
+ *
+ * <h4>ReadAhead Worker</h4>
+ * Most of readahead stats are exposed under scope `readahead_worker`. Only readahead exceptions are exposed
+ * in parent scope via <code>readAheadExceptionsLogger</code>.
+ * <ul>
+ * <li> `readahead_worker`/wait: counter. number of waits that readahead worker is waiting. If this keeps increasing,
+ * it usually means readahead keep getting full because of reader slows down reading.
+ * <li> `readahead_worker`/repositions: counter. number of repositions that readhead worker encounters. reposition
+ * means that a readahead worker finds that it isn't advancing to a new log segment and force re-positioning.
+ * <li> `readahead_worker`/entry_piggy_back_hits: counter. it increases when the last add confirmed being advanced
+ * because of the piggy-back lac.
+ * <li> `readahead_worker`/entry_piggy_back_misses: counter. it increases when the last add confirmed isn't advanced
+ * by a read entry because it doesn't piggy back a newer lac.
+ * <li> `readahead_worker`/read_entries: opstats. stats on number of entries read per readahead read batch.
+ * <li> `readahead_worker`/read_lac_counter: counter. stats on the number of readLastConfirmed operations
+ * <li> `readahead_worker`/read_lac_and_entry_counter: counter. stats on the number of readLastConfirmedAndEntry
+ * operations.
+ * <li> `readahead_worker`/cache_full: counter. it increases each time readahead worker finds cache become full.
+ * If it keeps increasing, that means reader slows down reading.
+ * <li> `readahead_worker`/resume: opstats. stats on readahead worker resuming reading from wait state.
+ * <li> `readahead_worker`/read_lac_lag: opstats. stats on the number of entries diff between the lac reader knew
+ * last time and the lac that it received. if `lag` between two subsequent lacs is high, that might means delay
+ * might be high. because reader is only allowed to read entries after lac is advanced.
+ * <li> `readahead_worker`/long_poll_interruption: opstats. stats on the number of interruptions happened to long
+ * poll. the interruptions are usually because of receiving zookeeper notifications.
+ * <li> `readahead_worker`/notification_execution: opstats. stats on executions over the notifications received from
+ * zookeeper.
+ * <li> `readahead_worker`/metadata_reinitialization: opstats. stats on metadata reinitialization after receiving
+ * notifcation from log segments updates.
+ * <li> `readahead_worker`/idle_reader_warn: counter. it increases each time the readahead worker detects itself
+ * becoming idle.
+ * </ul>
+ * <h4>Read Lock</h4>
+ * All read lock related stats are exposed under scope `read_lock`.
+ * for detail stats.
+ */
+class BKLogReadHandler extends BKLogHandler implements LogSegmentNamesListener {
+ static final Logger LOG = LoggerFactory.getLogger(BKLogReadHandler.class);
+
+ protected final LogMetadataForReader logMetadataForReader;
+
+ protected final DynamicDistributedLogConfiguration dynConf;
+
+ private final Optional<String> subscriberId;
+ private DistributedLock readLock;
+ private Future<Void> lockAcquireFuture;
+
+ // notify the state change about the read handler
+ protected final AsyncNotification readerStateNotification;
+
+ // log segments listener
+ protected boolean logSegmentsNotificationDisabled = false;
+ protected final CopyOnWriteArraySet<LogSegmentListener> listeners =
+ new CopyOnWriteArraySet<LogSegmentListener>();
+ protected Versioned<List<LogSegmentMetadata>> lastNotifiedLogSegments =
+ new Versioned<List<LogSegmentMetadata>>(null, Version.NEW);
+
+ // stats
+ private final StatsLogger perLogStatsLogger;
+
+ /**
+ * Construct a Bookkeeper journal manager.
+ */
+ BKLogReadHandler(LogMetadataForReader logMetadata,
+ Optional<String> subscriberId,
+ DistributedLogConfiguration conf,
+ DynamicDistributedLogConfiguration dynConf,
+ LogStreamMetadataStore streamMetadataStore,
+ LogSegmentMetadataCache metadataCache,
+ LogSegmentEntryStore entryStore,
+ OrderedScheduler scheduler,
+ AlertStatsLogger alertStatsLogger,
+ StatsLogger statsLogger,
+ StatsLogger perLogStatsLogger,
+ String clientId,
+ AsyncNotification readerStateNotification,
+ boolean isHandleForReading) {
+ super(logMetadata,
+ conf,
+ streamMetadataStore,
+ metadataCache,
+ entryStore,
+ scheduler,
+ statsLogger,
+ alertStatsLogger,
+ clientId);
+ this.logMetadataForReader = logMetadata;
+ this.dynConf = dynConf;
+ this.perLogStatsLogger =
+ isHandleForReading ? perLogStatsLogger : NullStatsLogger.INSTANCE;
+ this.readerStateNotification = readerStateNotification;
+ this.subscriberId = subscriberId;
+ }
+
+ @VisibleForTesting
+ String getReadLockPath() {
+ return logMetadataForReader.getReadLockPath(subscriberId);
+ }
+
+ <T> void satisfyPromiseAsync(final Promise<T> promise, final Try<T> result) {
+ scheduler.submit(new SafeRunnable() {
+ @Override
+ public void safeRun() {
+ promise.update(result);
+ }
+ });
+ }
+
+ Future<Void> checkLogStreamExists() {
+ return streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName());
+ }
+
+ /**
+ * Elective stream lock--readers are not required to acquire the lock before using the stream.
+ */
+ synchronized Future<Void> lockStream() {
+ if (null == lockAcquireFuture) {
+ lockAcquireFuture = streamMetadataStore.createReadLock(logMetadataForReader, subscriberId)
+ .flatMap(new ExceptionalFunction<DistributedLock, Future<Void>>() {
+ @Override
+ public Future<Void> applyE(DistributedLock lock) throws Throwable {
+ BKLogReadHandler.this.readLock = lock;
+ LOG.info("acquiring readlock {} at {}", getLockClientId(), getReadLockPath());
+ return acquireLockOnExecutorThread(lock);
+ }
+ });
+ }
+ return lockAcquireFuture;
+ }
+
+ /**
+ * Begin asynchronous lock acquire, but ensure that the returned future is satisfied on an
+ * executor service thread.
+ */
+ Future<Void> acquireLockOnExecutorThread(DistributedLock lock) throws LockingException {
+ final Future<? extends DistributedLock> acquireFuture = lock.asyncAcquire();
+
+ // The future we return must be satisfied on an executor service thread. If we simply
+ // return the future returned by asyncAcquire, user callbacks may end up running in
+ // the lock state executor thread, which will cause deadlocks and introduce latency
+ // etc.
+ final Promise<Void> threadAcquirePromise = new Promise<Void>();
+ threadAcquirePromise.setInterruptHandler(new Function<Throwable, BoxedUnit>() {
+ @Override
+ public BoxedUnit apply(Throwable t) {
+ FutureUtils.cancel(acquireFuture);
+ return null;
+ }
+ });
+ acquireFuture.addEventListener(new FutureEventListener<DistributedLock>() {
+ @Override
+ public void onSuccess(DistributedLock lock) {
+ LOG.info("acquired readlock {} at {}", getLockClientId(), getReadLockPath());
+ satisfyPromiseAsync(threadAcquirePromise, new Return<Void>(null));
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ LOG.info("failed to acquire readlock {} at {}",
+ new Object[]{ getLockClientId(), getReadLockPath(), cause });
+ satisfyPromiseAsync(threadAcquirePromise, new Throw<Void>(cause));
+ }
+ });
+ return threadAcquirePromise;
+ }
+
+ /**
+ * Check ownership of elective stream lock.
+ */
+ void checkReadLock() throws DLIllegalStateException, LockingException {
+ synchronized (this) {
+ if ((null == lockAcquireFuture) ||
+ (!lockAcquireFuture.isDefined())) {
+ throw new DLIllegalStateException("Attempt to check for lock before it has been acquired successfully");
+ }
+ }
+
+ readLock.checkOwnership();
+ }
+
+ public Future<Void> asyncClose() {
+ DistributedLock lockToClose;
+ synchronized (this) {
+ if (null != lockAcquireFuture && !lockAcquireFuture.isDefined()) {
+ FutureUtils.cancel(lockAcquireFuture);
+ }
+ lockToClose = readLock;
+ }
+ return Utils.closeSequence(scheduler, lockToClose)
+ .flatMap(new AbstractFunction1<Void, Future<Void>>() {
+ @Override
+ public Future<Void> apply(Void result) {
+ // unregister the log segment listener
+ metadataStore.unregisterLogSegmentListener(logMetadata.getLogSegmentsPath(), BKLogReadHandler.this);
+ return Future.Void();
+ }
+ });
+ }
+
+ @Override
+ public Future<Void> asyncAbort() {
+ return asyncClose();
+ }
+
+ /**
+ * Start fetch the log segments and register the {@link LogSegmentNamesListener}.
+ * The future is satisfied only on a successful fetch or encountered a fatal failure.
+ *
+ * @return future represents the fetch result
+ */
+ Future<Versioned<List<LogSegmentMetadata>>> asyncStartFetchLogSegments() {
+ Promise<Versioned<List<LogSegmentMetadata>>> promise =
+ new Promise<Versioned<List<LogSegmentMetadata>>>();
+ asyncStartFetchLogSegments(promise);
+ return promise;
+ }
+
+ void asyncStartFetchLogSegments(final Promise<Versioned<List<LogSegmentMetadata>>> promise) {
+ readLogSegmentsFromStore(
+ LogSegmentMetadata.COMPARATOR,
+ LogSegmentFilter.DEFAULT_FILTER,
+ this).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ if (cause instanceof LogNotFoundException ||
+ cause instanceof LogSegmentNotFoundException ||
+ cause instanceof UnexpectedException) {
+ // indicate some inconsistent behavior, abort
+ metadataException.compareAndSet(null, (IOException) cause);
+ // notify the reader that read handler is in error state
+ notifyReaderOnError(cause);
+ FutureUtils.setException(promise, cause);
+ return;
+ }
+ scheduler.schedule(new Runnable() {
+ @Override
+ public void run() {
+ asyncStartFetchLogSegments(promise);
+ }
+ }, conf.getZKRetryBackoffMaxMillis(), TimeUnit.MILLISECONDS);
+ }
+
+ @Override
+ public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
+ // no-op
+ FutureUtils.setValue(promise, segments);
+ }
+ });
+ }
+
+ @VisibleForTesting
+ void disableReadAheadLogSegmentsNotification() {
+ logSegmentsNotificationDisabled = true;
+ }
+
+ @Override
+ public void onSegmentsUpdated(final Versioned<List<String>> segments) {
+ synchronized (this) {
+ if (lastNotifiedLogSegments.getVersion() != Version.NEW &&
+ lastNotifiedLogSegments.getVersion().compare(segments.getVersion()) != Version.Occurred.BEFORE) {
+ // the log segments has been read, and it is possibly a retry from last segments update
+ return;
+ }
+ }
+
+ Promise<Versioned<List<LogSegmentMetadata>>> readLogSegmentsPromise =
+ new Promise<Versioned<List<LogSegmentMetadata>>>();
+ readLogSegmentsPromise.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ if (cause instanceof LogNotFoundException ||
+ cause instanceof LogSegmentNotFoundException ||
+ cause instanceof UnexpectedException) {
+ // indicate some inconsistent behavior, abort
+ metadataException.compareAndSet(null, (IOException) cause);
+ // notify the reader that read handler is in error state
+ notifyReaderOnError(cause);
+ return;
+ }
+ scheduler.schedule(new Runnable() {
+ @Override
+ public void run() {
+ onSegmentsUpdated(segments);
+ }
+ }, conf.getZKRetryBackoffMaxMillis(), TimeUnit.MILLISECONDS);
+ }
+
+ @Override
+ public void onSuccess(Versioned<List<LogSegmentMetadata>> logSegments) {
+ List<LogSegmentMetadata> segmentsToNotify = null;
+ synchronized (BKLogReadHandler.this) {
+ Versioned<List<LogSegmentMetadata>> lastLogSegments = lastNotifiedLogSegments;
+ if (lastLogSegments.getVersion() == Version.NEW ||
+ lastLogSegments.getVersion().compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
+ lastNotifiedLogSegments = logSegments;
+ segmentsToNotify = logSegments.getValue();
+ }
+ }
+ if (null != segmentsToNotify) {
+ notifyUpdatedLogSegments(segmentsToNotify);
+ }
+ }
+ });
+ // log segments list is updated, read their metadata
+ readLogSegmentsFromStore(
+ segments,
+ LogSegmentMetadata.COMPARATOR,
+ LogSegmentFilter.DEFAULT_FILTER,
+ readLogSegmentsPromise);
+ }
+
+ @Override
+ public void onLogStreamDeleted() {
+ notifyLogStreamDeleted();
+ }
+
+ //
+ // Listener for log segments
+ //
+
+ protected void registerListener(@Nullable LogSegmentListener listener) {
+ if (null != listener) {
+ listeners.add(listener);
+ }
+ }
+
+ protected void unregisterListener(@Nullable LogSegmentListener listener) {
+ if (null != listener) {
+ listeners.remove(listener);
+ }
+ }
+
+ protected void notifyUpdatedLogSegments(List<LogSegmentMetadata> segments) {
+ if (logSegmentsNotificationDisabled) {
+ return;
+ }
+
+ for (LogSegmentListener listener : listeners) {
+ List<LogSegmentMetadata> listToReturn =
+ new ArrayList<LogSegmentMetadata>(segments);
+ Collections.sort(listToReturn, LogSegmentMetadata.COMPARATOR);
+ listener.onSegmentsUpdated(listToReturn);
+ }
+ }
+
+ protected void notifyLogStreamDeleted() {
+ if (logSegmentsNotificationDisabled) {
+ return;
+ }
+
+ for (LogSegmentListener listener : listeners) {
+ listener.onLogStreamDeleted();
+ }
+ }
+
+ // notify the errors
+ protected void notifyReaderOnError(Throwable cause) {
+ if (null != readerStateNotification) {
+ readerStateNotification.notifyOnError(cause);
+ }
+ }
+}
[09/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
new file mode 100644
index 0000000..6b60c77
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
@@ -0,0 +1,1348 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.FlushException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
+import org.apache.distributedlog.exceptions.WriteCancelledException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
+import org.apache.distributedlog.feature.CoreFeatureKeys;
+import org.apache.distributedlog.injector.FailureInjector;
+import org.apache.distributedlog.injector.RandomDelayFailureInjector;
+import org.apache.distributedlog.io.Buffer;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.io.CompressionUtils;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
+import org.apache.distributedlog.logsegment.LogSegmentWriter;
+import org.apache.distributedlog.stats.BroadCastStatsLogger;
+import org.apache.distributedlog.stats.OpStatsListener;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.PermitLimiter;
+import org.apache.distributedlog.util.SafeQueueingFuturePool;
+import org.apache.distributedlog.util.SimplePermitLimiter;
+import org.apache.distributedlog.util.Sizable;
+import com.twitter.util.Function0;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.FuturePool;
+import com.twitter.util.Promise;
+import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.MathUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
+
+/**
+ * BookKeeper Based Log Segment Writer.
+ *
+ * Multiple log records are packed into a single bookkeeper
+ * entry before sending it over the network. The fact that the log record entries
+ * are complete in the bookkeeper entries means that each bookkeeper log entry
+ * can be read as a complete edit log. This is useful for reading, as we don't
+ * need to read through the entire log segment to get the last written entry.
+ *
+ * <h3>Metrics</h3>
+ *
+ * <ul>
+ * <li> flush/periodic/{success,miss}: counters for periodic flushes.
+ * <li> data/{success,miss}: counters for data transmits.
+ * <li> transmit/packetsize: opstats. characteristics of packet size for transmits.
+ * <li> control/success: counter of success transmit of control records
+ * <li> seg_writer/write: opstats. latency characteristics of write operations in segment writer.
+ * <li> seg_writer/add_complete/{callback,queued,deferred}: opstats. latency components of add completions.
+ * <li> seg_writer/pendings: counter. the number of records pending by the segment writers.
+ * <li> transmit/outstanding/requests: per stream gauge. the number of outstanding transmits each stream.
+ * </ul>
+ */
+class BKLogSegmentWriter implements LogSegmentWriter, AddCallback, Runnable, Sizable {
+ static final Logger LOG = LoggerFactory.getLogger(BKLogSegmentWriter.class);
+
+ private final String fullyQualifiedLogSegment;
+ private final String streamName;
+ private final int logSegmentMetadataVersion;
+ private BKTransmitPacket packetPrevious;
+ private Entry.Writer recordSetWriter;
+ private final AtomicInteger outstandingTransmits;
+ private final int transmissionThreshold;
+ protected final LogSegmentEntryWriter entryWriter;
+ private final CompressionCodec.Type compressionType;
+ private final ReentrantLock transmitLock = new ReentrantLock();
+ private final AtomicInteger transmitResult
+ = new AtomicInteger(BKException.Code.OK);
+ private final DistributedLock lock;
+ private final boolean isDurableWriteEnabled;
+ private DLSN lastDLSN = DLSN.InvalidDLSN;
+ private final long startTxId;
+ private long lastTxId = DistributedLogConstants.INVALID_TXID;
+ private long lastTxIdAcknowledged = DistributedLogConstants.INVALID_TXID;
+ private long outstandingBytes = 0;
+ private long numFlushesSinceRestart = 0;
+ private long numBytes = 0;
+ private long lastEntryId = Long.MIN_VALUE;
+ private long lastTransmitNanos = Long.MIN_VALUE;
+ private final int periodicKeepAliveMs;
+
+ // Indicates whether there are writes that have been successfully transmitted that would need
+ // a control record to be transmitted to make them visible to the readers by updating the last
+ // add confirmed
+ volatile private boolean controlFlushNeeded = false;
+ private boolean immediateFlushEnabled = false;
+ private int minDelayBetweenImmediateFlushMs = 0;
+ private Stopwatch lastTransmit;
+ private boolean streamEnded = false;
+ private final ScheduledFuture<?> periodicFlushSchedule;
+ private final ScheduledFuture<?> periodicKeepAliveSchedule;
+ final private AtomicReference<ScheduledFuture<?>> transmitSchedFutureRef = new AtomicReference<ScheduledFuture<?>>(null);
+ final private AtomicReference<ScheduledFuture<?>> immFlushSchedFutureRef = new AtomicReference<ScheduledFuture<?>>(null);
+ final private AtomicReference<Exception> scheduledFlushException = new AtomicReference<Exception>(null);
+ private boolean enforceLock = true;
+ private Promise<Void> closeFuture = null;
+ private final boolean enableRecordCounts;
+ private int positionWithinLogSegment = 0;
+ private final long logSegmentSequenceNumber;
+ // Used only for values that *could* change (e.g. buffer size etc.)
+ private final DistributedLogConfiguration conf;
+ private final OrderedScheduler scheduler;
+
+ // stats
+ private final StatsLogger envelopeStatsLogger;
+ private final StatsLogger transmitOutstandingLogger;
+ private final Counter transmitDataSuccesses;
+ private final Counter transmitDataMisses;
+ private final Gauge<Number> transmitOutstandingGauge;
+ private final OpStatsLogger transmitDataPacketSize;
+ private final Counter transmitControlSuccesses;
+ private final Counter pFlushSuccesses;
+ private final Counter pFlushMisses;
+ private final OpStatsLogger writeTime;
+ private final OpStatsLogger addCompleteTime;
+ private final OpStatsLogger addCompleteQueuedTime;
+ private final OpStatsLogger addCompleteDeferredTime;
+ private final Counter pendingWrites;
+
+ // add complete processing
+ private final SafeQueueingFuturePool<Void> addCompleteFuturePool;
+
+ // Functions
+ private final AbstractFunction1<Integer, Future<Long>> GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC =
+ new AbstractFunction1<Integer, Future<Long>>() {
+ @Override
+ public Future<Long> apply(Integer transmitRc) {
+ if (BKException.Code.OK == transmitRc) {
+ return Future.value(getLastTxIdAcknowledged());
+ } else {
+ return Future.exception(new BKTransmitException("Failed to transmit entry", transmitRc));
+ }
+ }
+ };
+ final AbstractFunction1<Long, Future<Long>> COMMIT_AFTER_FLUSH_FUNC =
+ new AbstractFunction1<Long, Future<Long>>() {
+ @Override
+ public Future<Long> apply(Long lastAckedTxId) {
+ return commit();
+ }
+ };
+
+ private final AlertStatsLogger alertStatsLogger;
+ private final WriteLimiter writeLimiter;
+ private final FailureInjector writeDelayInjector;
+
+ /**
+ * Construct an edit log output stream which writes to a ledger.
+ */
+ protected BKLogSegmentWriter(String streamName,
+ String logSegmentName,
+ DistributedLogConfiguration conf,
+ int logSegmentMetadataVersion,
+ LogSegmentEntryWriter entryWriter,
+ DistributedLock lock, /** the lock needs to be acquired **/
+ long startTxId,
+ long logSegmentSequenceNumber,
+ OrderedScheduler scheduler,
+ StatsLogger statsLogger,
+ StatsLogger perLogStatsLogger,
+ AlertStatsLogger alertStatsLogger,
+ PermitLimiter globalWriteLimiter,
+ FeatureProvider featureProvider,
+ DynamicDistributedLogConfiguration dynConf)
+ throws IOException {
+ super();
+
+ // set up a write limiter
+ PermitLimiter streamWriteLimiter = null;
+ if (conf.getPerWriterOutstandingWriteLimit() < 0) {
+ streamWriteLimiter = PermitLimiter.NULL_PERMIT_LIMITER;
+ } else {
+ Feature disableWriteLimitFeature = featureProvider.getFeature(
+ CoreFeatureKeys.DISABLE_WRITE_LIMIT.name().toLowerCase());
+ streamWriteLimiter = new SimplePermitLimiter(
+ conf.getOutstandingWriteLimitDarkmode(),
+ conf.getPerWriterOutstandingWriteLimit(),
+ statsLogger.scope("streamWriteLimiter"),
+ false,
+ disableWriteLimitFeature);
+ }
+ this.writeLimiter = new WriteLimiter(streamName, streamWriteLimiter, globalWriteLimiter);
+ this.alertStatsLogger = alertStatsLogger;
+ this.envelopeStatsLogger = BroadCastStatsLogger.masterslave(statsLogger, perLogStatsLogger);
+
+ StatsLogger flushStatsLogger = statsLogger.scope("flush");
+ StatsLogger pFlushStatsLogger = flushStatsLogger.scope("periodic");
+ pFlushSuccesses = pFlushStatsLogger.getCounter("success");
+ pFlushMisses = pFlushStatsLogger.getCounter("miss");
+
+ // transmit
+ StatsLogger transmitDataStatsLogger = statsLogger.scope("data");
+ transmitDataSuccesses = transmitDataStatsLogger.getCounter("success");
+ transmitDataMisses = transmitDataStatsLogger.getCounter("miss");
+ StatsLogger transmitStatsLogger = statsLogger.scope("transmit");
+ transmitDataPacketSize = transmitStatsLogger.getOpStatsLogger("packetsize");
+ StatsLogger transmitControlStatsLogger = statsLogger.scope("control");
+ transmitControlSuccesses = transmitControlStatsLogger.getCounter("success");
+ StatsLogger segWriterStatsLogger = statsLogger.scope("seg_writer");
+ writeTime = segWriterStatsLogger.getOpStatsLogger("write");
+ addCompleteTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("callback");
+ addCompleteQueuedTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("queued");
+ addCompleteDeferredTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("deferred");
+ pendingWrites = segWriterStatsLogger.getCounter("pending");
+
+ // outstanding transmit requests
+ transmitOutstandingLogger = perLogStatsLogger.scope("transmit").scope("outstanding");
+ transmitOutstandingGauge = new Gauge<Number>() {
+ @Override
+ public Number getDefaultValue() {
+ return 0;
+ }
+ @Override
+ public Number getSample() {
+ return outstandingTransmits.get();
+ }
+ };
+ transmitOutstandingLogger.registerGauge("requests", transmitOutstandingGauge);
+
+ outstandingTransmits = new AtomicInteger(0);
+ this.fullyQualifiedLogSegment = streamName + ":" + logSegmentName;
+ this.streamName = streamName;
+ this.logSegmentMetadataVersion = logSegmentMetadataVersion;
+ this.entryWriter = entryWriter;
+ this.lock = lock;
+ this.lock.checkOwnershipAndReacquire();
+
+ final int configuredTransmissionThreshold = dynConf.getOutputBufferSize();
+ if (configuredTransmissionThreshold > MAX_LOGRECORDSET_SIZE) {
+ LOG.warn("Setting output buffer size {} greater than max transmission size {} for log segment {}",
+ new Object[] {configuredTransmissionThreshold, MAX_LOGRECORDSET_SIZE, fullyQualifiedLogSegment});
+ this.transmissionThreshold = MAX_LOGRECORDSET_SIZE;
+ } else {
+ this.transmissionThreshold = configuredTransmissionThreshold;
+ }
+ this.compressionType = CompressionUtils.stringToType(conf.getCompressionType());
+
+ this.logSegmentSequenceNumber = logSegmentSequenceNumber;
+ this.recordSetWriter = Entry.newEntry(
+ streamName,
+ Math.max(transmissionThreshold, 1024),
+ envelopeBeforeTransmit(),
+ compressionType,
+ envelopeStatsLogger);
+ this.packetPrevious = null;
+ this.startTxId = startTxId;
+ this.lastTxId = startTxId;
+ this.lastTxIdAcknowledged = startTxId;
+ this.enableRecordCounts = conf.getEnableRecordCounts();
+ this.immediateFlushEnabled = conf.getImmediateFlushEnabled();
+ this.isDurableWriteEnabled = dynConf.isDurableWriteEnabled();
+ this.scheduler = scheduler;
+
+ // Failure injection
+ if (conf.getEIInjectWriteDelay()) {
+ this.writeDelayInjector = new RandomDelayFailureInjector(dynConf);
+ } else {
+ this.writeDelayInjector = FailureInjector.NULL;
+ }
+
+ // If we are transmitting immediately (threshold == 0) and if immediate
+ // flush is enabled, we don't need the periodic flush task
+ final int configuredPeriodicFlushFrequency = dynConf.getPeriodicFlushFrequencyMilliSeconds();
+ if (!immediateFlushEnabled || (0 != this.transmissionThreshold)) {
+ int periodicFlushFrequency = configuredPeriodicFlushFrequency;
+ if (periodicFlushFrequency > 0 && scheduler != null) {
+ periodicFlushSchedule = scheduler.scheduleAtFixedRate(this,
+ periodicFlushFrequency/2, periodicFlushFrequency/2, TimeUnit.MILLISECONDS);
+ } else {
+ periodicFlushSchedule = null;
+ }
+ } else {
+ // Min delay heuristic applies only when immediate flush is enabled
+ // and transmission threshold is zero
+ minDelayBetweenImmediateFlushMs = conf.getMinDelayBetweenImmediateFlushMs();
+ periodicFlushSchedule = null;
+ }
+ this.periodicKeepAliveMs = conf.getPeriodicKeepAliveMilliSeconds();
+ if (periodicKeepAliveMs > 0 && scheduler != null) {
+ periodicKeepAliveSchedule = scheduler.scheduleAtFixedRate(new Runnable() {
+ @Override
+ public void run() {
+ keepAlive();
+ }
+ }, periodicKeepAliveMs, periodicKeepAliveMs, TimeUnit.MILLISECONDS);
+ } else {
+ periodicKeepAliveSchedule = null;
+ }
+
+ this.conf = conf;
+ if (null != scheduler) {
+ this.addCompleteFuturePool = new SafeQueueingFuturePool<Void>(scheduler.getFuturePool(streamName));
+ } else {
+ this.addCompleteFuturePool = null;
+ }
+ assert(!this.immediateFlushEnabled || (null != this.scheduler));
+ this.lastTransmit = Stopwatch.createStarted();
+ }
+
+ String getFullyQualifiedLogSegment() {
+ return fullyQualifiedLogSegment;
+ }
+
+ @VisibleForTesting
+ DistributedLock getLock() {
+ return this.lock;
+ }
+
+ @VisibleForTesting
+ FuturePool getFuturePool() {
+ if (null == scheduler) {
+ return null;
+ }
+ return scheduler.getFuturePool(streamName);
+ }
+
+ @VisibleForTesting
+ void setTransmitResult(int rc) {
+ transmitResult.set(rc);
+ }
+
+ @VisibleForTesting
+ protected final LogSegmentEntryWriter getEntryWriter() {
+ return this.entryWriter;
+ }
+
+ @Override
+ public long getLogSegmentId() {
+ return this.entryWriter.getLogSegmentId();
+ }
+
+ protected final long getLogSegmentSequenceNumber() {
+ return logSegmentSequenceNumber;
+ }
+
+ /**
+ * Get the start tx id of the log segment.
+ *
+ * @return start tx id of the log segment.
+ */
+ protected final long getStartTxId() {
+ return startTxId;
+ }
+
+ /**
+ * Get the last tx id that has been written to the log segment buffer but not committed yet.
+ *
+ * @return last tx id that has been written to the log segment buffer but not committed yet.
+ * @see #getLastTxIdAcknowledged()
+ */
+ synchronized long getLastTxId() {
+ return lastTxId;
+ }
+
+ /**
+ * Get the last tx id that has been acknowledged.
+ *
+ * @return last tx id that has been acknowledged.
+ * @see #getLastTxId()
+ */
+ synchronized long getLastTxIdAcknowledged() {
+ return lastTxIdAcknowledged;
+ }
+
+ /**
+ * Get the position-within-logsemgnet of the last written log record.
+ *
+ * @return position-within-logsegment of the last written log record.
+ */
+ synchronized int getPositionWithinLogSegment() {
+ return positionWithinLogSegment;
+ }
+
+ @VisibleForTesting
+ long getLastEntryId() {
+ return lastEntryId;
+ }
+
+ /**
+ * Get the last dlsn of the last acknowledged record.
+ *
+ * @return last dlsn of the last acknowledged record.
+ */
+ synchronized DLSN getLastDLSN() {
+ return lastDLSN;
+ }
+
+ @Override
+ public long size() {
+ return entryWriter.size();
+ }
+
+ private synchronized int getAverageTransmitSize() {
+ if (numFlushesSinceRestart > 0) {
+ long ret = numBytes/numFlushesSinceRestart;
+
+ if (ret < Integer.MIN_VALUE || ret > Integer.MAX_VALUE) {
+ throw new IllegalArgumentException
+ (ret + " transmit size should never exceed max transmit size");
+ }
+ return (int) ret;
+ }
+
+ return 0;
+ }
+
+ private Entry.Writer newRecordSetWriter() {
+ return Entry.newEntry(
+ streamName,
+ Math.max(transmissionThreshold, getAverageTransmitSize()),
+ envelopeBeforeTransmit(),
+ compressionType,
+ envelopeStatsLogger);
+ }
+
+ private boolean envelopeBeforeTransmit() {
+ return LogSegmentMetadata.supportsEnvelopedEntries(logSegmentMetadataVersion);
+ }
+
+ @Override
+ public Future<Void> asyncClose() {
+ return closeInternal(false);
+ }
+
+ @Override
+ public Future<Void> asyncAbort() {
+ return closeInternal(true);
+ }
+
+ private void flushAddCompletes() {
+ if (null != addCompleteFuturePool) {
+ addCompleteFuturePool.close();
+ }
+ }
+
+ private synchronized void abortPacket(BKTransmitPacket packet) {
+ long numRecords = 0;
+ if (null != packet) {
+ EntryBuffer recordSet = packet.getRecordSet();
+ numRecords = recordSet.getNumRecords();
+ int rc = transmitResult.get();
+ if (BKException.Code.OK == rc) {
+ rc = BKException.Code.InterruptedException;
+ }
+ Throwable reason = new WriteCancelledException(streamName, FutureUtils.transmitException(rc));
+ recordSet.abortTransmit(reason);
+ }
+ LOG.info("Stream {} aborted {} writes", fullyQualifiedLogSegment, numRecords);
+ }
+
+ private synchronized long getWritesPendingTransmit() {
+ if (null != recordSetWriter) {
+ return recordSetWriter.getNumRecords();
+ } else {
+ return 0;
+ }
+ }
+
+ private synchronized long getPendingAddCompleteCount() {
+ if (null != addCompleteFuturePool) {
+ return addCompleteFuturePool.size();
+ } else {
+ return 0;
+ }
+ }
+
+ private Future<Void> closeInternal(boolean abort) {
+ Promise<Void> closePromise;
+ synchronized (this) {
+ if (null != closeFuture) {
+ return closeFuture;
+ }
+ closePromise = closeFuture = new Promise<Void>();
+ }
+
+ AtomicReference<Throwable> throwExc = new AtomicReference<Throwable>(null);
+ closeInternal(abort, throwExc, closePromise);
+ return closePromise;
+ }
+
+ private void closeInternal(final boolean abort,
+ final AtomicReference<Throwable> throwExc,
+ final Promise<Void> closePromise) {
+ // clean stats resources
+ this.transmitOutstandingLogger.unregisterGauge("requests", transmitOutstandingGauge);
+ this.writeLimiter.close();
+
+ // Cancel the periodic keep alive schedule first
+ if (null != periodicKeepAliveSchedule) {
+ if (!periodicKeepAliveSchedule.cancel(false)) {
+ LOG.info("Periodic keepalive for log segment {} isn't cancelled.", getFullyQualifiedLogSegment());
+ }
+ }
+
+ // Cancel the periodic flush schedule first
+ // The task is allowed to exit gracefully
+ if (null != periodicFlushSchedule) {
+ // we don't need to care about the cancel result here. if the periodicl flush task couldn't
+ // be cancelled, it means that it is doing flushing. So following flushes would be synchronized
+ // to wait until background flush completed.
+ if (!periodicFlushSchedule.cancel(false)) {
+ LOG.info("Periodic flush for log segment {} isn't cancelled.", getFullyQualifiedLogSegment());
+ }
+ }
+
+ // If it is a normal close and the stream isn't in an error state, we attempt to flush any buffered data
+ if (!abort && !isLogSegmentInError()) {
+ this.enforceLock = false;
+ LOG.info("Flushing before closing log segment {}", getFullyQualifiedLogSegment());
+ flushAndCommit().addEventListener(new FutureEventListener<Long>() {
+ @Override
+ public void onSuccess(Long value) {
+ abortTransmitPacketOnClose(abort, throwExc, closePromise);
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ throwExc.set(cause);
+ abortTransmitPacketOnClose(abort, throwExc, closePromise);
+ }
+ });
+ } else {
+ abortTransmitPacketOnClose(abort, throwExc, closePromise);
+ }
+
+ }
+
+ private void abortTransmitPacketOnClose(final boolean abort,
+ final AtomicReference<Throwable> throwExc,
+ final Promise<Void> closePromise) {
+ LOG.info("Closing BKPerStreamLogWriter (abort={}) for {} :" +
+ " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {} addCompletesPending = {}",
+ new Object[]{abort, fullyQualifiedLogSegment, getLastDLSN(),
+ outstandingTransmits.get(), getWritesPendingTransmit(), getPendingAddCompleteCount()});
+
+ // Save the current packet to reset, leave a new empty packet to avoid a race with
+ // addCompleteDeferredProcessing.
+ final BKTransmitPacket packetPreviousSaved;
+ final BKTransmitPacket packetCurrentSaved;
+ synchronized (this) {
+ packetPreviousSaved = packetPrevious;
+ packetCurrentSaved = new BKTransmitPacket(recordSetWriter);
+ recordSetWriter = newRecordSetWriter();
+ }
+
+ // Once the last packet been transmitted, apply any remaining promises asynchronously
+ // to avoid blocking close if bk client is slow for some reason.
+ if (null != packetPreviousSaved) {
+ packetPreviousSaved.addTransmitCompleteListener(new FutureEventListener<Integer>() {
+ @Override
+ public void onSuccess(Integer transmitResult) {
+ flushAddCompletes();
+ abortPacket(packetCurrentSaved);
+ }
+ @Override
+ public void onFailure(Throwable cause) {
+ LOG.error("Unexpected error on transmit completion ", cause);
+ }
+ });
+ } else {
+ // In this case there are no pending add completes, but we still need to abort the
+ // current packet.
+ abortPacket(packetCurrentSaved);
+ }
+ closeLedgerOnClose(abort, throwExc, closePromise);
+ }
+
+ private void closeLedgerOnClose(final boolean abort,
+ final AtomicReference<Throwable> throwExc,
+ final Promise<Void> closePromise) {
+ // close the log segment if it isn't in error state, so all the outstanding addEntry(s) will callback.
+ if (null == throwExc.get() && !isLogSegmentInError()) {
+ // Synchronous closing the ledger handle, if we couldn't close a ledger handle successfully.
+ // we should throw the exception to #closeToFinalize, so it would fail completing a log segment.
+ entryWriter.asyncClose(new CloseCallback() {
+ @Override
+ public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
+ if (BKException.Code.OK != rc && BKException.Code.LedgerClosedException != rc) {
+ if (!abort) {
+ throwExc.set(new IOException("Failed to close ledger for " + fullyQualifiedLogSegment + " : " +
+ BKException.getMessage(rc)));
+ }
+ }
+ completeClosePromise(abort, throwExc, closePromise);
+ }
+ }, null);
+ } else {
+ completeClosePromise(abort, throwExc, closePromise);
+ }
+ }
+
+ private void completeClosePromise(final boolean abort,
+ final AtomicReference<Throwable> throwExc,
+ final Promise<Void> closePromise) {
+ // If add entry failed because of closing ledger above, we don't need to fail the close operation
+ if (!abort && null == throwExc.get() && shouldFailCompleteLogSegment()) {
+ throwExc.set(new BKTransmitException("Closing an errored stream : ", transmitResult.get()));
+ }
+
+ if (null == throwExc.get()) {
+ FutureUtils.setValue(closePromise, null);
+ } else {
+ FutureUtils.setException(closePromise, throwExc.get());
+ }
+ }
+
+ @Override
+ synchronized public void write(LogRecord record) throws IOException {
+ writeUserRecord(record);
+ flushIfNeeded();
+ }
+
+ @Override
+ synchronized public Future<DLSN> asyncWrite(LogRecord record) {
+ return asyncWrite(record, true);
+ }
+
+ synchronized public Future<DLSN> asyncWrite(LogRecord record, boolean flush) {
+ Future<DLSN> result = null;
+ try {
+ if (record.isControl()) {
+ // we don't pack control records with user records together
+ // so transmit current output buffer if possible
+ try {
+ transmit();
+ } catch (IOException ioe) {
+ return Future.exception(new WriteCancelledException(fullyQualifiedLogSegment, ioe));
+ }
+ result = writeControlLogRecord(record);
+ transmit();
+ } else {
+ result = writeUserRecord(record);
+ if (!isDurableWriteEnabled) {
+ // we have no idea about the DLSN if durability is turned off.
+ result = Future.value(DLSN.InvalidDLSN);
+ }
+ if (flush) {
+ flushIfNeeded();
+ }
+ }
+ } catch (IOException ioe) {
+ // We may incorrectly report transmit failure here, but only if we happened to hit
+ // packet/xmit size limit conditions AND fail flush above, which should happen rarely
+ if (null != result) {
+ LOG.error("Overriding first result with flush failure {}", result);
+ }
+ result = Future.exception(ioe);
+
+ // Flush to ensure any prev. writes with flush=false are flushed despite failure.
+ flushIfNeededNoThrow();
+ }
+ return result;
+ }
+
+ synchronized private Future<DLSN> writeUserRecord(LogRecord record) throws IOException {
+ if (null != closeFuture) {
+ throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(BKException.Code.LedgerClosedException));
+ }
+
+ if (BKException.Code.OK != transmitResult.get()) {
+ // Failfast if the stream already encountered error with safe retry on the client
+ throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(transmitResult.get()));
+ }
+
+ if (streamEnded) {
+ throw new EndOfStreamException("Writing to a stream after it has been marked as completed");
+ }
+
+ if ((record.getTransactionId() < 0) ||
+ (record.getTransactionId() == DistributedLogConstants.MAX_TXID)) {
+ throw new TransactionIdOutOfOrderException(record.getTransactionId());
+ }
+
+ // Inject write delay if configured to do so
+ writeDelayInjector.inject();
+
+ // Will check write rate limits and throw if exceeded.
+ writeLimiter.acquire();
+ pendingWrites.inc();
+
+ // The count represents the number of user records up to the
+ // current record
+ // Increment the record count only when writing a user log record
+ // Internally generated log records don't increment the count
+ // writeInternal will always set a count regardless of whether it was
+ // incremented or not.
+ Future<DLSN> future = null;
+ try {
+ // increment the position for the record to write
+ // if the record is failed to write, it would be decremented.
+ positionWithinLogSegment++;
+ int numRecords = 1;
+ if (record.isRecordSet()) {
+ numRecords = LogRecordSet.numRecords(record);
+ }
+ future = writeInternal(record);
+ // after the record (record set) is written, the position should be
+ // moved for {numRecords}, but since we already moved the record by 1
+ // so advance the position for other {numRecords - 1}.
+ positionWithinLogSegment += (numRecords - 1);
+ } catch (IOException ex) {
+ writeLimiter.release();
+ pendingWrites.dec();
+ positionWithinLogSegment--;
+ throw ex;
+ }
+
+ // Track outstanding requests and return the future.
+ return future.ensure(new Function0<BoxedUnit>() {
+ public BoxedUnit apply() {
+ pendingWrites.dec();
+ writeLimiter.release();
+ return null;
+ }
+ });
+ }
+
+ boolean isLogSegmentInError() {
+ return (transmitResult.get() != BKException.Code.OK);
+ }
+
+ boolean shouldFailCompleteLogSegment() {
+ return (transmitResult.get() != BKException.Code.OK) &&
+ (transmitResult.get() != BKException.Code.LedgerClosedException);
+ }
+
+ synchronized public Future<DLSN> writeInternal(LogRecord record)
+ throws LogRecordTooLongException, LockingException, BKTransmitException,
+ WriteException, InvalidEnvelopedEntryException {
+ int logRecordSize = record.getPersistentSize();
+
+ if (logRecordSize > MAX_LOGRECORD_SIZE) {
+ throw new LogRecordTooLongException(String.format(
+ "Log Record of size %d written when only %d is allowed",
+ logRecordSize, MAX_LOGRECORD_SIZE));
+ }
+
+ // If we will exceed the max number of bytes allowed per entry
+ // initiate a transmit before accepting the new log record
+ if ((recordSetWriter.getNumBytes() + logRecordSize) > MAX_LOGRECORDSET_SIZE) {
+ checkStateAndTransmit();
+ }
+
+ checkWriteLock();
+
+ if (enableRecordCounts) {
+ // Set the count here. The caller would appropriately increment it
+ // if this log record is to be counted
+ record.setPositionWithinLogSegment(positionWithinLogSegment);
+ }
+
+ Promise<DLSN> writePromise = new Promise<DLSN>();
+ writePromise.addEventListener(new OpStatsListener<DLSN>(writeTime));
+ recordSetWriter.writeRecord(record, writePromise);
+
+ if (record.getTransactionId() < lastTxId) {
+ LOG.info("Log Segment {} TxId decreased Last: {} Record: {}",
+ new Object[] {fullyQualifiedLogSegment, lastTxId, record.getTransactionId()});
+ }
+ if (!record.isControl()) {
+ // only update last tx id for user records
+ lastTxId = record.getTransactionId();
+ outstandingBytes += (20 + record.getPayload().length);
+ }
+ return writePromise;
+ }
+
+ synchronized private Future<DLSN> writeControlLogRecord()
+ throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
+ LockingException, LogRecordTooLongException {
+ LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.CONTROL_RECORD_CONTENT);
+ controlRec.setControl();
+ return writeControlLogRecord(controlRec);
+ }
+
+ synchronized private Future<DLSN> writeControlLogRecord(LogRecord record)
+ throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
+ LockingException, LogRecordTooLongException {
+ return writeInternal(record);
+ }
+
+ /**
+ * We write a special log record that marks the end of the stream. Since this is the last
+ * log record in the stream, it is marked with MAX_TXID. MAX_TXID also has the useful
+ * side-effect of disallowing future startLogSegment calls through the MaxTxID check
+ *
+ * @throws IOException
+ */
+ synchronized private void writeEndOfStreamMarker() throws IOException {
+ LogRecord endOfStreamRec = new LogRecord(DistributedLogConstants.MAX_TXID, "endOfStream".getBytes(UTF_8));
+ endOfStreamRec.setEndOfStream();
+ writeInternal(endOfStreamRec);
+ }
+
+ /**
+ * Flushes all the data up to this point,
+ * adds the end of stream marker and marks the stream
+ * as read-only in the metadata. No appends to the
+ * stream will be allowed after this point
+ */
+ public Future<Long> markEndOfStream() {
+ synchronized (this) {
+ try {
+ writeEndOfStreamMarker();
+ } catch (IOException e) {
+ return Future.exception(e);
+ }
+ streamEnded = true;
+ }
+ return flushAndCommit();
+ }
+
+ /**
+ * Write bulk of records.
+ *
+ * (TODO: moved this method to log writer level)
+ *
+ * @param records list of records to write
+ * @return number of records that has been written
+ * @throws IOException when there is I/O errors during writing records.
+ */
+ synchronized public int writeBulk(List<LogRecord> records) throws IOException {
+ int numRecords = 0;
+ for (LogRecord r : records) {
+ write(r);
+ numRecords++;
+ }
+ return numRecords;
+ }
+
+ private void checkStateBeforeTransmit() throws WriteException {
+ try {
+ FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitBeforeAddEntry);
+ } catch (IOException e) {
+ throw new WriteException(streamName, "Fail transmit before adding entries");
+ }
+ }
+
+ /**
+ * Transmit the output buffer data to the backend.
+ *
+ * @return last txn id that already acknowledged
+ * @throws BKTransmitException if the segment writer is already in error state
+ * @throws LockingException if the segment writer lost lock before transmit
+ * @throws WriteException if failed to create the envelope for the data to transmit
+ * @throws InvalidEnvelopedEntryException when built an invalid enveloped entry
+ */
+ synchronized void checkStateAndTransmit()
+ throws BKTransmitException, WriteException, InvalidEnvelopedEntryException, LockingException {
+ checkStateBeforeTransmit();
+ transmit();
+ }
+
+ @Override
+ public synchronized Future<Long> flush() {
+ try {
+ checkStateBeforeTransmit();
+ } catch (WriteException e) {
+ return Future.exception(e);
+ }
+
+ Future<Integer> transmitFuture;
+ try {
+ transmitFuture = transmit();
+ } catch (BKTransmitException e) {
+ return Future.exception(e);
+ } catch (LockingException e) {
+ return Future.exception(e);
+ } catch (WriteException e) {
+ return Future.exception(e);
+ } catch (InvalidEnvelopedEntryException e) {
+ return Future.exception(e);
+ }
+
+ if (null == transmitFuture) {
+ if (null != packetPrevious) {
+ transmitFuture = packetPrevious.getTransmitFuture();
+ } else {
+ return Future.value(getLastTxIdAcknowledged());
+ }
+ }
+
+ return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
+ }
+
+ @Override
+ public synchronized Future<Long> commit() {
+ // we don't pack control records with user records together
+ // so transmit current output buffer if possible
+ Future<Integer> transmitFuture;
+ try {
+ try {
+ transmitFuture = transmit();
+ } catch (IOException ioe) {
+ return Future.exception(ioe);
+ }
+ if (null == transmitFuture) {
+ writeControlLogRecord();
+ return flush();
+ }
+ } catch (IOException ioe) {
+ return Future.exception(ioe);
+ }
+ return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
+ }
+
+ Future<Long> flushAndCommit() {
+ return flush().flatMap(COMMIT_AFTER_FLUSH_FUNC);
+ }
+
+ void flushIfNeededNoThrow() {
+ try {
+ flushIfNeeded();
+ } catch (IOException ioe) {
+ LOG.error("Encountered exception while flushing log records to stream {}",
+ fullyQualifiedLogSegment, ioe);
+ }
+ }
+
+ void scheduleFlushWithDelayIfNeeded(final Callable<?> callable,
+ final AtomicReference<ScheduledFuture<?>> scheduledFutureRef) {
+ final long delayMs = Math.max(0, minDelayBetweenImmediateFlushMs - lastTransmit.elapsed(TimeUnit.MILLISECONDS));
+ final ScheduledFuture<?> scheduledFuture = scheduledFutureRef.get();
+ if ((null == scheduledFuture) || scheduledFuture.isDone()) {
+ scheduledFutureRef.set(scheduler.schedule(new Runnable() {
+ @Override
+ public void run() {
+ synchronized(this) {
+ scheduledFutureRef.set(null);
+ try {
+ callable.call();
+
+ // Flush was successful or wasn't needed, the exception should be unset.
+ scheduledFlushException.set(null);
+ } catch (Exception exc) {
+ scheduledFlushException.set(exc);
+ LOG.error("Delayed flush failed", exc);
+ }
+ }
+ }
+ }, delayMs, TimeUnit.MILLISECONDS));
+ }
+ }
+
+ // Based on transmit buffer size, immediate flush, etc., should we flush the current
+ // packet now.
+ void flushIfNeeded() throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
+ LockingException, FlushException {
+ if (outstandingBytes > transmissionThreshold) {
+ // If flush delay is disabled, flush immediately, else schedule appropriately.
+ if (0 == minDelayBetweenImmediateFlushMs) {
+ checkStateAndTransmit();
+ } else {
+ scheduleFlushWithDelayIfNeeded(new Callable<Void>() {
+ @Override
+ public Void call() throws Exception {
+ checkStateAndTransmit();
+ return null;
+ }
+ }, transmitSchedFutureRef);
+
+ // Timing here is not very important--the last flush failed and we should
+ // indicate this to the caller. The next flush may succeed and unset the
+ // scheduledFlushException in which case the next write will succeed (if the caller
+ // hasn't already closed the writer).
+ if (scheduledFlushException.get() != null) {
+ throw new FlushException("Last flush encountered an error while writing data to the backend",
+ getLastTxId(), getLastTxIdAcknowledged(), scheduledFlushException.get());
+ }
+ }
+ }
+ }
+
+ private void checkWriteLock() throws LockingException {
+ try {
+ if (FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_WriteInternalLostLock)) {
+ throw new LockingException("/failpoint/lockpath", "failpoint is simulating a lost lock"
+ + getFullyQualifiedLogSegment());
+ }
+ } catch (IOException e) {
+ throw new LockingException("/failpoint/lockpath", "failpoint is simulating a lost lock for "
+ + getFullyQualifiedLogSegment());
+ }
+ if (enforceLock) {
+ lock.checkOwnershipAndReacquire();
+ }
+ }
+
+ /**
+ * Transmit the current buffer to bookkeeper.
+ * Synchronised at the class. #write() and #setReadyToFlush()
+ * are never called at the same time.
+ *
+ * NOTE: This method should only throw known exceptions so that we don't accidentally
+ * add new code that throws in an inappropriate place.
+ *
+ * @return a transmit future for caller to wait for transmit result if we transmit successfully,
+ * null if no data to transmit
+ * @throws BKTransmitException if the segment writer is already in error state
+ * @throws LockingException if the segment writer lost lock before transmit
+ * @throws WriteException if failed to create the envelope for the data to transmit
+ * @throws InvalidEnvelopedEntryException when built an invalid enveloped entry
+ */
+ private Future<Integer> transmit()
+ throws BKTransmitException, LockingException, WriteException, InvalidEnvelopedEntryException {
+ EntryBuffer recordSetToTransmit;
+ transmitLock.lock();
+ try {
+ synchronized (this) {
+ checkWriteLock();
+ // If transmitResult is anything other than BKException.Code.OK, it means that the
+ // stream has encountered an error and cannot be written to.
+ if (!transmitResult.compareAndSet(BKException.Code.OK,
+ BKException.Code.OK)) {
+ LOG.error("Log Segment {} Trying to write to an errored stream; Error is {}",
+ fullyQualifiedLogSegment,
+ BKException.getMessage(transmitResult.get()));
+ throw new BKTransmitException("Trying to write to an errored stream;"
+ + " Error code : (" + transmitResult.get()
+ + ") " + BKException.getMessage(transmitResult.get()), transmitResult.get());
+ }
+
+ if (recordSetWriter.getNumRecords() == 0) {
+ // Control flushes always have at least the control record to flush
+ transmitDataMisses.inc();
+ return null;
+ }
+
+ recordSetToTransmit = recordSetWriter;
+ recordSetWriter = newRecordSetWriter();
+ outstandingBytes = 0;
+
+ if (recordSetToTransmit.hasUserRecords()) {
+ numBytes += recordSetToTransmit.getNumBytes();
+ numFlushesSinceRestart++;
+ }
+ }
+
+ Buffer toSend;
+ try {
+ toSend = recordSetToTransmit.getBuffer();
+ FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitFailGetBuffer);
+ } catch (IOException e) {
+ if (e instanceof InvalidEnvelopedEntryException) {
+ alertStatsLogger.raise("Invalid enveloped entry for segment {} : ", fullyQualifiedLogSegment, e);
+ }
+ LOG.error("Exception while enveloping entries for segment: {}",
+ new Object[] {fullyQualifiedLogSegment}, e);
+ // If a write fails here, we need to set the transmit result to an error so that
+ // no future writes go through and violate ordering guarantees.
+ transmitResult.set(BKException.Code.WriteException);
+ if (e instanceof InvalidEnvelopedEntryException) {
+ alertStatsLogger.raise("Invalid enveloped entry for segment {} : ", fullyQualifiedLogSegment, e);
+ throw (InvalidEnvelopedEntryException) e;
+ } else {
+ throw new WriteException(streamName, "Envelope Error");
+ }
+ }
+
+ synchronized (this) {
+ // update the transmit timestamp
+ lastTransmitNanos = MathUtils.nowInNano();
+
+ BKTransmitPacket packet = new BKTransmitPacket(recordSetToTransmit);
+ packetPrevious = packet;
+ entryWriter.asyncAddEntry(toSend.getData(), 0, toSend.size(),
+ this, packet);
+
+ if (recordSetToTransmit.hasUserRecords()) {
+ transmitDataSuccesses.inc();
+ } else {
+ transmitControlSuccesses.inc();
+ }
+
+ lastTransmit.reset().start();
+ outstandingTransmits.incrementAndGet();
+ controlFlushNeeded = false;
+ return packet.getTransmitFuture();
+ }
+ } finally {
+ transmitLock.unlock();
+ }
+ }
+
+ /**
+ * Checks if there is any data to transmit so that the periodic flush
+ * task can determine if there is anything it needs to do
+ */
+ synchronized private boolean haveDataToTransmit() {
+ if (!transmitResult.compareAndSet(BKException.Code.OK, BKException.Code.OK)) {
+ // Even if there is data it cannot be transmitted, so effectively nothing to send
+ return false;
+ }
+
+ return (recordSetWriter.getNumRecords() > 0);
+ }
+
+ @Override
+ public void addComplete(final int rc, LedgerHandle handle,
+ final long entryId, final Object ctx) {
+ final AtomicReference<Integer> effectiveRC = new AtomicReference<Integer>(rc);
+ try {
+ if (FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitComplete)) {
+ effectiveRC.set(BKException.Code.UnexpectedConditionException);
+ }
+ } catch (Exception exc) {
+ effectiveRC.set(BKException.Code.UnexpectedConditionException);
+ }
+
+ // Sanity check to make sure we're receiving these callbacks in order.
+ if (entryId > -1 && lastEntryId >= entryId) {
+ LOG.error("Log segment {} saw out of order entry {} lastEntryId {}",
+ new Object[] {fullyQualifiedLogSegment, entryId, lastEntryId});
+ }
+ lastEntryId = entryId;
+
+ assert (ctx instanceof BKTransmitPacket);
+ final BKTransmitPacket transmitPacket = (BKTransmitPacket) ctx;
+
+ // Time from transmit until receipt of addComplete callback
+ addCompleteTime.registerSuccessfulEvent(TimeUnit.MICROSECONDS.convert(
+ System.nanoTime() - transmitPacket.getTransmitTime(), TimeUnit.NANOSECONDS));
+
+ if (BKException.Code.OK == rc) {
+ EntryBuffer recordSet = transmitPacket.getRecordSet();
+ if (recordSet.hasUserRecords()) {
+ synchronized (this) {
+ lastTxIdAcknowledged = Math.max(lastTxIdAcknowledged, recordSet.getMaxTxId());
+ }
+ }
+ }
+
+ if (null != addCompleteFuturePool) {
+ final Stopwatch queuedTime = Stopwatch.createStarted();
+ addCompleteFuturePool.apply(new Function0<Void>() {
+ public Void apply() {
+ final Stopwatch deferredTime = Stopwatch.createStarted();
+ addCompleteQueuedTime.registerSuccessfulEvent(queuedTime.elapsed(TimeUnit.MICROSECONDS));
+ addCompleteDeferredProcessing(transmitPacket, entryId, effectiveRC.get());
+ addCompleteDeferredTime.registerSuccessfulEvent(deferredTime.elapsed(TimeUnit.MICROSECONDS));
+ return null;
+ }
+ @Override
+ public String toString() {
+ return String.format("AddComplete(Stream=%s, entryId=%d, rc=%d)",
+ fullyQualifiedLogSegment, entryId, rc);
+ }
+ }).addEventListener(new FutureEventListener<Void>() {
+ @Override
+ public void onSuccess(Void done) {
+ }
+ @Override
+ public void onFailure(Throwable cause) {
+ LOG.error("addComplete processing failed for {} entry {} lastTxId {} rc {} with error",
+ new Object[] {fullyQualifiedLogSegment, entryId, transmitPacket.getRecordSet().getMaxTxId(), rc, cause});
+ }
+ });
+ // Race condition if we notify before the addComplete is enqueued.
+ transmitPacket.notifyTransmitComplete(effectiveRC.get());
+ outstandingTransmits.getAndDecrement();
+ } else {
+ // Notify transmit complete must be called before deferred processing in the
+ // sync case since otherwise callbacks in deferred processing may deadlock.
+ transmitPacket.notifyTransmitComplete(effectiveRC.get());
+ outstandingTransmits.getAndDecrement();
+ addCompleteDeferredProcessing(transmitPacket, entryId, effectiveRC.get());
+ }
+ }
+
+ private void addCompleteDeferredProcessing(final BKTransmitPacket transmitPacket,
+ final long entryId,
+ final int rc) {
+ boolean cancelPendingPromises = false;
+ EntryBuffer recordSet = transmitPacket.getRecordSet();
+ synchronized (this) {
+ if (transmitResult.compareAndSet(BKException.Code.OK, rc)) {
+ // If this is the first time we are setting an error code in the transmitResult then
+ // we must cancel pending promises; once this error has been set, more records will not
+ // be enqueued; they will be failed with WriteException
+ cancelPendingPromises = (BKException.Code.OK != rc);
+ } else {
+ LOG.warn("Log segment {} entryId {}: Tried to set transmit result to ({}) but is already ({})",
+ new Object[] {fullyQualifiedLogSegment, entryId, rc, transmitResult.get()});
+ }
+
+ if (transmitResult.get() != BKException.Code.OK) {
+ if (recordSet.hasUserRecords()) {
+ transmitDataPacketSize.registerFailedEvent(recordSet.getNumBytes());
+ }
+ } else {
+ // If we had data that we flushed then we need it to make sure that
+ // background flush in the next pass will make the previous writes
+ // visible by advancing the lastAck
+ if (recordSet.hasUserRecords()) {
+ transmitDataPacketSize.registerSuccessfulEvent(recordSet.getNumBytes());
+ controlFlushNeeded = true;
+ if (immediateFlushEnabled) {
+ if (0 == minDelayBetweenImmediateFlushMs) {
+ backgroundFlush(true);
+ } else {
+ scheduleFlushWithDelayIfNeeded(new Callable<Void>() {
+ @Override
+ public Void call() throws Exception {
+ backgroundFlush(true);
+ return null;
+ }
+ }, immFlushSchedFutureRef);
+ }
+ }
+ }
+ }
+
+ // update last dlsn before satisifying future
+ if (BKException.Code.OK == transmitResult.get()) {
+ DLSN lastDLSNInPacket = recordSet.finalizeTransmit(
+ logSegmentSequenceNumber, entryId);
+ if (recordSet.hasUserRecords()) {
+ if (null != lastDLSNInPacket && lastDLSN.compareTo(lastDLSNInPacket) < 0) {
+ lastDLSN = lastDLSNInPacket;
+ }
+ }
+ }
+ }
+
+ if (BKException.Code.OK == transmitResult.get()) {
+ recordSet.completeTransmit(logSegmentSequenceNumber, entryId);
+ } else {
+ recordSet.abortTransmit(FutureUtils.transmitException(transmitResult.get()));
+ }
+
+ if (cancelPendingPromises) {
+ // Since the writer is in a bad state no more packets will be tramsitted, and its safe to
+ // assign a new empty packet. This is to avoid a race with closeInternal which may also
+ // try to cancel the current packet;
+ final BKTransmitPacket packetCurrentSaved;
+ synchronized (this) {
+ packetCurrentSaved = new BKTransmitPacket(recordSetWriter);
+ recordSetWriter = newRecordSetWriter();
+ }
+ packetCurrentSaved.getRecordSet().abortTransmit(
+ new WriteCancelledException(streamName,
+ FutureUtils.transmitException(transmitResult.get())));
+ }
+ }
+
+ @Override
+ synchronized public void run() {
+ backgroundFlush(false);
+ }
+
+ synchronized private void backgroundFlush(boolean controlFlushOnly) {
+ if (null != closeFuture) {
+ // if the log segment is closing, skip any background flushing
+ LOG.debug("Skip background flushing since log segment {} is closing.", getFullyQualifiedLogSegment());
+ return;
+ }
+ try {
+ boolean newData = haveDataToTransmit();
+
+ if (controlFlushNeeded || (!controlFlushOnly && newData)) {
+ // If we need this periodic transmit to persist previously written data but
+ // there is no new data (which would cause the transmit to be skipped) generate
+ // a control record
+ if (!newData) {
+ writeControlLogRecord();
+ }
+
+ transmit();
+ pFlushSuccesses.inc();
+ } else {
+ pFlushMisses.inc();
+ }
+ } catch (IOException exc) {
+ LOG.error("Log Segment {}: Error encountered by the periodic flush", fullyQualifiedLogSegment, exc);
+ }
+ }
+
+ synchronized private void keepAlive() {
+ if (null != closeFuture) {
+ // if the log segment is closing, skip sending any keep alive records.
+ LOG.debug("Skip sending keepAlive control record since log segment {} is closing.",
+ getFullyQualifiedLogSegment());
+ return;
+ }
+
+ if (MathUtils.elapsedMSec(lastTransmitNanos) < periodicKeepAliveMs) {
+ return;
+ }
+
+ LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.KEEPALIVE_RECORD_CONTENT);
+ controlRec.setControl();
+ asyncWrite(controlRec);
+ }
+
+}
[27/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClient.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClient.java
deleted file mode 100644
index 74cd6cf..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClient.java
+++ /dev/null
@@ -1,402 +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.Stopwatch;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.zk.ZKWatcherManager;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.Watcher.Event.EventType;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.ACL;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * ZooKeeper Client wrapper over {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}.
- * It handles retries on session expires and provides a watcher manager {@link ZKWatcherManager}.
- *
- * <h3>Metrics</h3>
- * <ul>
- * <li> zookeeper operation stats are exposed under scope <code>zk</code> by
- * {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}
- * <li> stats on zookeeper watched events are exposed under scope <code>watcher</code> by
- * {@link org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase}
- * <li> stats about {@link ZKWatcherManager} are exposed under scope <code>watcher_manager</code>
- * </ul>
- */
-public class ZooKeeperClient {
-
- public static interface Credentials {
-
- Credentials NONE = new Credentials() {
- @Override
- public void authenticate(ZooKeeper zooKeeper) {
- // noop
- }
- };
-
- void authenticate(ZooKeeper zooKeeper);
- }
-
- public static class DigestCredentials implements Credentials {
-
- String username;
- String password;
-
- public DigestCredentials(String username, String password) {
- this.username = username;
- this.password = password;
- }
-
- @Override
- public void authenticate(ZooKeeper zooKeeper) {
- zooKeeper.addAuthInfo("digest", String.format("%s:%s", username, password).getBytes(UTF_8));
- }
- }
-
- public interface ZooKeeperSessionExpireNotifier {
- void notifySessionExpired();
- }
-
- /**
- * Indicates an error connecting to a zookeeper cluster.
- */
- public static class ZooKeeperConnectionException extends IOException {
- private static final long serialVersionUID = 6682391687004819361L;
-
- public ZooKeeperConnectionException(String message) {
- super(message);
- }
-
- public ZooKeeperConnectionException(String message, Throwable cause) {
- super(message, cause);
- }
- }
-
- private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClient.class.getName());
-
- private final String name;
- private final int sessionTimeoutMs;
- private final int defaultConnectionTimeoutMs;
- private final String zooKeeperServers;
- // GuardedBy "this", but still volatile for tests, where we want to be able to see writes
- // made from within long synchronized blocks.
- private volatile ZooKeeper zooKeeper = null;
- private final RetryPolicy retryPolicy;
- private final StatsLogger statsLogger;
- private final int retryThreadCount;
- private final double requestRateLimit;
- private final Credentials credentials;
- private volatile boolean authenticated = false;
- private Stopwatch disconnectedStopwatch = null;
-
- private boolean closed = false;
-
- final Set<Watcher> watchers = new CopyOnWriteArraySet<Watcher>();
-
- // watcher manager to manage watchers
- private final ZKWatcherManager watcherManager;
-
- /**
- * Creates an unconnected client that will lazily attempt to connect on the first call to
- * {@link #get}. All successful connections will be authenticated with the given
- * {@code credentials}.
- *
- * @param sessionTimeoutMs
- * ZK session timeout in milliseconds
- * @param connectionTimeoutMs
- * ZK connection timeout in milliseconds
- * @param zooKeeperServers
- * the set of servers forming the ZK cluster
- */
- ZooKeeperClient(int sessionTimeoutMs, int connectionTimeoutMs, String zooKeeperServers) {
- this("default", sessionTimeoutMs, connectionTimeoutMs, zooKeeperServers, null, NullStatsLogger.INSTANCE, 1, 0,
- Credentials.NONE);
- }
-
- ZooKeeperClient(String name,
- int sessionTimeoutMs,
- int connectionTimeoutMs,
- String zooKeeperServers,
- RetryPolicy retryPolicy,
- StatsLogger statsLogger,
- int retryThreadCount,
- double requestRateLimit,
- Credentials credentials) {
- this.name = name;
- this.sessionTimeoutMs = sessionTimeoutMs;
- this.zooKeeperServers = zooKeeperServers;
- this.defaultConnectionTimeoutMs = connectionTimeoutMs;
- this.retryPolicy = retryPolicy;
- this.statsLogger = statsLogger;
- this.retryThreadCount = retryThreadCount;
- this.requestRateLimit = requestRateLimit;
- this.credentials = credentials;
- this.watcherManager = ZKWatcherManager.newBuilder()
- .name(name)
- .zkc(this)
- .statsLogger(statsLogger.scope("watcher_manager"))
- .build();
- }
-
- public List<ACL> getDefaultACL() {
- if (Credentials.NONE == credentials) {
- return ZooDefs.Ids.OPEN_ACL_UNSAFE;
- } else {
- return DistributedLogConstants.EVERYONE_READ_CREATOR_ALL;
- }
- }
-
- public ZKWatcherManager getWatcherManager() {
- return watcherManager;
- }
-
- /**
- * Returns the current active ZK connection or establishes a new one if none has yet been
- * established or a previous connection was disconnected or had its session time out.
- *
- * @return a connected ZooKeeper client
- * @throws ZooKeeperConnectionException if there was a problem connecting to the ZK cluster
- * @throws InterruptedException if interrupted while waiting for a connection to be established
- * @throws TimeoutException if a connection could not be established within the configured
- * session timeout
- */
- public synchronized ZooKeeper get()
- throws ZooKeeperConnectionException, InterruptedException {
-
- try {
- FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
- } catch (IOException ioe) {
- throw new ZooKeeperConnectionException("Client " + name + " failed on establishing zookeeper connection", ioe);
- }
-
- // This indicates that the client was explictly closed
- if (closed) {
- throw new ZooKeeperConnectionException("Client " + name + " has already been closed");
- }
-
- // the underneath zookeeper is retryable zookeeper
- if (zooKeeper != null && retryPolicy != null) {
- if (zooKeeper.getState().equals(ZooKeeper.States.CONNECTED)) {
- // the zookeeper client is connected
- disconnectedStopwatch = null;
- } else {
- if (disconnectedStopwatch == null) {
- disconnectedStopwatch = Stopwatch.createStarted();
- } else {
- long disconnectedMs = disconnectedStopwatch.elapsed(TimeUnit.MILLISECONDS);
- if (disconnectedMs > defaultConnectionTimeoutMs) {
- closeInternal();
- authenticated = false;
- }
- }
- }
- }
-
- if (zooKeeper == null) {
- zooKeeper = buildZooKeeper();
- disconnectedStopwatch = null;
- }
-
- // In case authenticate throws an exception, the caller can try to recover the client by
- // calling get again.
- if (!authenticated) {
- credentials.authenticate(zooKeeper);
- authenticated = true;
- }
-
- return zooKeeper;
- }
-
- private ZooKeeper buildZooKeeper()
- throws ZooKeeperConnectionException, InterruptedException {
- Watcher watcher = new Watcher() {
- @Override
- public void process(WatchedEvent event) {
- switch (event.getType()) {
- case None:
- switch (event.getState()) {
- case Expired:
- if (null == retryPolicy) {
- LOG.info("ZooKeeper {}' session expired. Event: {}", name, event);
- closeInternal();
- }
- authenticated = false;
- break;
- case Disconnected:
- if (null == retryPolicy) {
- LOG.info("ZooKeeper {} is disconnected from zookeeper now," +
- " but it is OK unless we received EXPIRED event.", name);
- }
- // Mark as not authenticated if expired or disconnected. In both cases
- // we lose any attached auth info. Relying on Expired/Disconnected is
- // sufficient since all Expired/Disconnected events are processed before
- // all SyncConnected events, and the underlying member is not updated until
- // SyncConnected is received.
- authenticated = false;
- break;
- default:
- break;
- }
- }
-
- try {
- for (Watcher watcher : watchers) {
- try {
- watcher.process(event);
- } catch (Throwable t) {
- LOG.warn("Encountered unexpected exception from watcher {} : ", watcher, t);
- }
- }
- } catch (Throwable t) {
- LOG.warn("Encountered unexpected exception when firing watched event {} : ", event, t);
- }
- }
- };
-
- Set<Watcher> watchers = new HashSet<Watcher>();
- watchers.add(watcher);
-
- ZooKeeper zk;
- try {
- RetryPolicy opRetryPolicy = null == retryPolicy ?
- new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) : retryPolicy;
- RetryPolicy connectRetryPolicy = null == retryPolicy ?
- new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) :
- new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, Integer.MAX_VALUE);
- zk = org.apache.bookkeeper.zookeeper.ZooKeeperClient.newBuilder()
- .connectString(zooKeeperServers)
- .sessionTimeoutMs(sessionTimeoutMs)
- .watchers(watchers)
- .operationRetryPolicy(opRetryPolicy)
- .connectRetryPolicy(connectRetryPolicy)
- .statsLogger(statsLogger)
- .retryThreadCount(retryThreadCount)
- .requestRateLimit(requestRateLimit)
- .build();
- } catch (KeeperException e) {
- throw new ZooKeeperConnectionException("Problem connecting to servers: " + zooKeeperServers, e);
- } catch (IOException e) {
- throw new ZooKeeperConnectionException("Problem connecting to servers: " + zooKeeperServers, e);
- }
- return zk;
- }
-
- /**
- * Clients that need to re-establish state after session expiration can register an
- * {@code onExpired} command to execute.
- *
- * @param onExpired the {@code Command} to register
- * @return the new {@link Watcher} which can later be passed to {@link #unregister} for
- * removal.
- */
- public Watcher registerExpirationHandler(final ZooKeeperSessionExpireNotifier onExpired) {
- Watcher watcher = new Watcher() {
- @Override
- public void process(WatchedEvent event) {
- if (event.getType() == EventType.None && event.getState() == KeeperState.Expired) {
- try {
- onExpired.notifySessionExpired();
- } catch (Exception exc) {
- // do nothing
- }
- }
- }
- };
- register(watcher);
- return watcher;
- }
-
- /**
- * Clients that need to register a top-level {@code Watcher} should do so using this method. The
- * registered {@code watcher} will remain registered across re-connects and session expiration
- * events.
- *
- * @param watcher the {@code Watcher to register}
- */
- public void register(Watcher watcher) {
- if (null != watcher) {
- watchers.add(watcher);
- }
- }
-
- /**
- * Clients can attempt to unregister a top-level {@code Watcher} that has previously been
- * registered.
- *
- * @param watcher the {@code Watcher} to unregister as a top-level, persistent watch
- * @return whether the given {@code Watcher} was found and removed from the active set
- */
- public boolean unregister(Watcher watcher) {
- return null != watcher && watchers.remove(watcher);
- }
-
- /**
- * Closes the current connection if any expiring the current ZooKeeper session. Any subsequent
- * calls to this method will no-op until the next successful {@link #get}.
- */
- public synchronized void closeInternal() {
- if (zooKeeper != null) {
- try {
- LOG.info("Closing zookeeper client {}.", name);
- zooKeeper.close();
- LOG.info("Closed zookeeper client {}.", name);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- LOG.warn("Interrupted trying to close zooKeeper {} : ", name, e);
- } finally {
- zooKeeper = null;
- }
- }
- }
-
- /**
- * Closes the the underlying zookeeper instance.
- * Subsequent attempts to {@link #get} will fail
- */
- public synchronized void close() {
- if (closed) {
- return;
- }
- LOG.info("Close zookeeper client {}.", name);
- closeInternal();
- // unregister gauges to prevent GC spiral
- this.watcherManager.unregisterGauges();
- closed = true;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
deleted file mode 100644
index 15f1805..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
+++ /dev/null
@@ -1,233 +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.Preconditions;
-import com.twitter.distributedlog.ZooKeeperClient.Credentials;
-import com.twitter.distributedlog.ZooKeeperClient.DigestCredentials;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-
-/**
- * Builder to build zookeeper client.
- */
-public class ZooKeeperClientBuilder {
-
- static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClientBuilder.class);
-
- /**
- * Create a zookeeper client builder to build zookeeper clients.
- *
- * @return zookeeper client builder.
- */
- public static ZooKeeperClientBuilder newBuilder() {
- return new ZooKeeperClientBuilder();
- }
-
- // name
- private String name = "default";
- // sessionTimeoutMs
- private int sessionTimeoutMs = -1;
- // conectionTimeoutMs
- private int conectionTimeoutMs = -1;
- // zkServers
- private String zkServers = null;
- // retry policy
- private RetryPolicy retryPolicy = null;
- // stats logger
- private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
- // retry executor thread count
- private int retryThreadCount = 1;
- // zookeeper access requestRateLimit limit
- private double requestRateLimit = 0;
- // Did call the zkAclId setter on the builder, used to ensure the setter is set.
- private boolean zkAclIdSet = false;
- private String zkAclId;
-
- // Cached ZooKeeper Client
- private ZooKeeperClient cachedClient = null;
-
- private ZooKeeperClientBuilder() {}
-
- /**
- * Set zookeeper client name
- *
- * @param name zookeeper client name
- * @return zookeeper client builder
- */
- public synchronized ZooKeeperClientBuilder name(String name) {
- this.name = name;
- return this;
- }
-
- /**
- * Set zookeeper session timeout in milliseconds.
- *
- * @param sessionTimeoutMs
- * session timeout in milliseconds.
- * @return zookeeper client builder.
- */
- public synchronized ZooKeeperClientBuilder sessionTimeoutMs(int sessionTimeoutMs) {
- this.sessionTimeoutMs = sessionTimeoutMs;
- if (this.conectionTimeoutMs <= 0) {
- this.conectionTimeoutMs = 2 * sessionTimeoutMs;
- }
- return this;
- }
-
- public synchronized ZooKeeperClientBuilder retryThreadCount(int retryThreadCount) {
- this.retryThreadCount = retryThreadCount;
- return this;
- }
-
- public synchronized ZooKeeperClientBuilder requestRateLimit(double requestRateLimit) {
- this.requestRateLimit = requestRateLimit;
- return this;
- }
-
- /**
- * Set zookeeper connection timeout in milliseconds
- *
- * @param connectionTimeoutMs
- * connection timeout ms.
- * @return builder
- */
- public synchronized ZooKeeperClientBuilder connectionTimeoutMs(int connectionTimeoutMs) {
- this.conectionTimeoutMs = connectionTimeoutMs;
- return this;
- }
-
- /**
- * Set ZooKeeper Connect String.
- *
- * @param zkServers
- * zookeeper servers to connect.
- * @return builder
- */
- public synchronized ZooKeeperClientBuilder zkServers(String zkServers) {
- this.zkServers = zkServers;
- return this;
- }
-
- /**
- * Set DistributedLog URI.
- *
- * @param uri
- * distributedlog uri.
- * @return builder.
- */
- public synchronized ZooKeeperClientBuilder uri(URI uri) {
- this.zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
- return this;
- }
-
- /**
- * Build zookeeper client using existing <i>zkc</i> client.
- *
- * @param zkc
- * zookeeper client.
- * @return builder
- */
- public synchronized ZooKeeperClientBuilder zkc(ZooKeeperClient zkc) {
- this.cachedClient = zkc;
- return this;
- }
-
- /**
- * Build zookeeper client with given retry policy <i>retryPolicy</i>.
- *
- * @param retryPolicy
- * retry policy
- * @return builder
- */
- public synchronized ZooKeeperClientBuilder retryPolicy(RetryPolicy retryPolicy) {
- this.retryPolicy = retryPolicy;
- return this;
- }
-
- /**
- * Build zookeeper client with given stats logger <i>statsLogger</i>.
- *
- * @param statsLogger
- * stats logger to expose zookeeper stats
- * @return builder
- */
- public synchronized ZooKeeperClientBuilder statsLogger(StatsLogger statsLogger) {
- this.statsLogger = statsLogger;
- return this;
- }
-
- /**
- * * Build zookeeper client with given zk acl digest id <i>zkAclId</i>.
- */
- public synchronized ZooKeeperClientBuilder zkAclId(String zkAclId) {
- this.zkAclIdSet = true;
- this.zkAclId = zkAclId;
- return this;
- }
-
- private void validateParameters() {
- Preconditions.checkNotNull(zkServers, "No zk servers provided.");
- Preconditions.checkArgument(conectionTimeoutMs > 0,
- "Invalid connection timeout : %d", conectionTimeoutMs);
- Preconditions.checkArgument(sessionTimeoutMs > 0,
- "Invalid session timeout : %d", sessionTimeoutMs);
- Preconditions.checkNotNull(statsLogger, "No stats logger provided.");
- Preconditions.checkArgument(zkAclIdSet, "Zookeeper acl id not set.");
- }
-
- /**
- * Build a zookeeper client.
- *
- * @return zookeeper client.
- */
- public synchronized ZooKeeperClient build() {
- if (null == cachedClient) {
- cachedClient = buildClient();
- }
- return cachedClient;
- }
-
- private ZooKeeperClient buildClient() {
- validateParameters();
-
- Credentials credentials = Credentials.NONE;
- if (null != zkAclId) {
- credentials = new DigestCredentials(zkAclId, zkAclId);
- }
-
- return new ZooKeeperClient(
- name,
- sessionTimeoutMs,
- conectionTimeoutMs,
- zkServers,
- retryPolicy,
- statsLogger,
- retryThreadCount,
- requestRateLimit,
- credentials
- );
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/AccessControlManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/AccessControlManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/AccessControlManager.java
deleted file mode 100644
index 5fcc87e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/AccessControlManager.java
+++ /dev/null
@@ -1,74 +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.acl;
-
-/**
- * Access Control on stream operations
- */
-public interface AccessControlManager {
-
- /**
- * Whether allowing writing to a stream.
- *
- * @param stream
- * Stream to write
- * @return true if allowing writing to the given stream, otherwise false.
- */
- boolean allowWrite(String stream);
-
- /**
- * Whether allowing truncating a given stream.
- *
- * @param stream
- * Stream to truncate
- * @return true if allowing truncating a given stream.
- */
- boolean allowTruncate(String stream);
-
- /**
- * Whether allowing deleting a given stream.
- *
- * @param stream
- * Stream to delete
- * @return true if allowing deleting a given stream.
- */
- boolean allowDelete(String stream);
-
- /**
- * Whether allowing proxies to acquire a given stream.
- *
- * @param stream
- * stream to acquire
- * @return true if allowing proxies to acquire the given stream.
- */
- boolean allowAcquire(String stream);
-
- /**
- * Whether allowing proxies to release ownership for a given stream.
- *
- * @param stream
- * stream to release
- * @return true if allowing proxies to release a given stream.
- */
- boolean allowRelease(String stream);
-
- /**
- * Close the access control manager.
- */
- void close();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/DefaultAccessControlManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/DefaultAccessControlManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/DefaultAccessControlManager.java
deleted file mode 100644
index e757595..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/DefaultAccessControlManager.java
+++ /dev/null
@@ -1,55 +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.acl;
-
-public class DefaultAccessControlManager implements AccessControlManager {
-
- public static final DefaultAccessControlManager INSTANCE = new DefaultAccessControlManager();
-
- private DefaultAccessControlManager() {
- }
-
- @Override
- public boolean allowWrite(String stream) {
- return true;
- }
-
- @Override
- public boolean allowTruncate(String stream) {
- return true;
- }
-
- @Override
- public boolean allowDelete(String stream) {
- return true;
- }
-
- @Override
- public boolean allowAcquire(String stream) {
- return true;
- }
-
- @Override
- public boolean allowRelease(String stream) {
- return true;
- }
-
- @Override
- public void close() {
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/package-info.java
deleted file mode 100644
index 65109fc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Access Control for distributedlog streams.
- */
-package com.twitter.distributedlog.acl;
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 0512907..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
+++ /dev/null
@@ -1,921 +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.admin;
-
-import com.google.common.base.Preconditions;
-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.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.logsegment.LogSegmentEntryStore;
-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.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-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;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Admin Tool for DistributedLog.
- */
-public class DistributedLogAdmin extends DistributedLogTool {
-
- static final Logger LOG = LoggerFactory.getLogger(DistributedLogAdmin.class);
-
- /**
- * Fix inprogress segment with lower ledger sequence number.
- *
- * @param namespace
- * dl namespace
- * @param metadataUpdater
- * metadata updater.
- * @param streamName
- * stream name.
- * @param verbose
- * print verbose messages.
- * @param interactive
- * is confirmation needed before executing actual action.
- * @throws IOException
- */
- public static void fixInprogressSegmentWithLowerSequenceNumber(final DistributedLogNamespace namespace,
- final MetadataUpdater metadataUpdater,
- final String streamName,
- final boolean verbose,
- final boolean interactive) throws IOException {
- DistributedLogManager dlm = namespace.openLog(streamName);
- try {
- List<LogSegmentMetadata> segments = dlm.getLogSegments();
- if (verbose) {
- System.out.println("LogSegments for " + streamName + " : ");
- for (LogSegmentMetadata segment : segments) {
- System.out.println(segment.getLogSegmentSequenceNumber() + "\t: " + segment);
- }
- }
- LOG.info("Get log segments for {} : {}", streamName, segments);
- // validate log segments
- long maxCompletedLogSegmentSequenceNumber = -1L;
- LogSegmentMetadata inprogressSegment = null;
- for (LogSegmentMetadata segment : segments) {
- if (!segment.isInProgress()) {
- maxCompletedLogSegmentSequenceNumber = Math.max(maxCompletedLogSegmentSequenceNumber, segment.getLogSegmentSequenceNumber());
- } else {
- // we already found an inprogress segment
- if (null != inprogressSegment) {
- throw new DLIllegalStateException("Multiple inprogress segments found for stream " + streamName + " : " + segments);
- }
- inprogressSegment = segment;
- }
- }
- if (null == inprogressSegment || inprogressSegment.getLogSegmentSequenceNumber() > maxCompletedLogSegmentSequenceNumber) {
- // nothing to fix
- return;
- }
- final long newLogSegmentSequenceNumber = maxCompletedLogSegmentSequenceNumber + 1;
- if (interactive && !IOUtils.confirmPrompt("Confirm to fix (Y/N), Ctrl+C to break : ")) {
- return;
- }
- final LogSegmentMetadata newSegment =
- FutureUtils.result(metadataUpdater.changeSequenceNumber(inprogressSegment, newLogSegmentSequenceNumber));
- LOG.info("Fixed {} : {} -> {} ",
- new Object[] { streamName, inprogressSegment, newSegment });
- if (verbose) {
- System.out.println("Fixed " + streamName + " : " + inprogressSegment.getZNodeName()
- + " -> " + newSegment.getZNodeName());
- System.out.println("\t old: " + inprogressSegment);
- System.out.println("\t new: " + newSegment);
- System.out.println();
- }
- } finally {
- dlm.close();
- }
- }
-
- private static class LogSegmentCandidate {
- final LogSegmentMetadata metadata;
- final LogRecordWithDLSN lastRecord;
-
- LogSegmentCandidate(LogSegmentMetadata metadata, LogRecordWithDLSN lastRecord) {
- this.metadata = metadata;
- this.lastRecord = lastRecord;
- }
-
- @Override
- public String toString() {
- return "LogSegmentCandidate[ metadata = " + metadata + ", last record = " + lastRecord + " ]";
- }
-
- }
-
- private static final Comparator<LogSegmentCandidate> LOG_SEGMENT_CANDIDATE_COMPARATOR =
- new Comparator<LogSegmentCandidate>() {
- @Override
- public int compare(LogSegmentCandidate o1, LogSegmentCandidate o2) {
- return LogSegmentMetadata.COMPARATOR.compare(o1.metadata, o2.metadata);
- }
- };
-
- private static class StreamCandidate {
-
- final String streamName;
- final SortedSet<LogSegmentCandidate> segmentCandidates =
- new TreeSet<LogSegmentCandidate>(LOG_SEGMENT_CANDIDATE_COMPARATOR);
-
- StreamCandidate(String streamName) {
- this.streamName = streamName;
- }
-
- synchronized void addLogSegmentCandidate(LogSegmentCandidate segmentCandidate) {
- segmentCandidates.add(segmentCandidate);
- }
-
- @Override
- public String toString() {
- return "StreamCandidate[ name = " + streamName + ", segments = " + segmentCandidates + " ]";
- }
- }
-
- public static void checkAndRepairDLNamespace(final URI uri,
- final DistributedLogNamespace namespace,
- final MetadataUpdater metadataUpdater,
- final OrderedScheduler scheduler,
- final boolean verbose,
- final boolean interactive) throws IOException {
- checkAndRepairDLNamespace(uri, namespace, metadataUpdater, scheduler, verbose, interactive, 1);
- }
-
- public static void checkAndRepairDLNamespace(final URI uri,
- final DistributedLogNamespace namespace,
- final MetadataUpdater metadataUpdater,
- final OrderedScheduler scheduler,
- 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.
- 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 under " + uri);
- }
- if (streams.size() == 0) {
- System.out.println("+ 0. nothing to check. quit.");
- return;
- }
- Map<String, StreamCandidate> streamCandidates =
- checkStreams(namespace, streams, scheduler, concurrency);
- if (verbose) {
- System.out.println("+ 0. " + streamCandidates.size() + " corrupted streams found.");
- }
- if (interactive && !IOUtils.confirmPrompt("Do you want to fix all " + streamCandidates.size() + " corrupted streams (Y/N) : ")) {
- return;
- }
- if (verbose) {
- System.out.println("- 1. repairing " + streamCandidates.size() + " corrupted streams.");
- }
- for (StreamCandidate candidate : streamCandidates.values()) {
- if (!repairStream(metadataUpdater, candidate, verbose, interactive)) {
- if (verbose) {
- System.out.println("* 1. aborted repairing corrupted streams.");
- }
- return;
- }
- }
- if (verbose) {
- System.out.println("+ 1. repaired " + streamCandidates.size() + " corrupted streams.");
- }
- }
-
- private static Map<String, StreamCandidate> checkStreams(
- final DistributedLogNamespace namespace,
- final Collection<String> streams,
- final OrderedScheduler scheduler,
- final int concurrency) throws IOException {
- final LinkedBlockingQueue<String> streamQueue =
- new LinkedBlockingQueue<String>();
- streamQueue.addAll(streams);
- final Map<String, StreamCandidate> candidateMap =
- new ConcurrentSkipListMap<String, StreamCandidate>();
- final AtomicInteger numPendingStreams = new AtomicInteger(streams.size());
- final CountDownLatch doneLatch = new CountDownLatch(1);
- Runnable checkRunnable = new Runnable() {
- @Override
- public void run() {
- while (!streamQueue.isEmpty()) {
- String stream;
- try {
- stream = streamQueue.take();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- break;
- }
- StreamCandidate candidate;
- try {
- LOG.info("Checking stream {}.", stream);
- candidate = checkStream(namespace, stream, scheduler);
- LOG.info("Checked stream {} - {}.", stream, candidate);
- } catch (IOException e) {
- LOG.error("Error on checking stream {} : ", stream, e);
- doneLatch.countDown();
- break;
- }
- if (null != candidate) {
- candidateMap.put(stream, candidate);
- }
- if (numPendingStreams.decrementAndGet() == 0) {
- doneLatch.countDown();
- }
- }
- }
- };
- Thread[] threads = new Thread[concurrency];
- for (int i = 0; i < concurrency; i++) {
- threads[i] = new Thread(checkRunnable, "check-thread-" + i);
- threads[i].start();
- }
- try {
- doneLatch.await();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- if (numPendingStreams.get() != 0) {
- throw new IOException(numPendingStreams.get() + " streams left w/o checked");
- }
- for (int i = 0; i < concurrency; i++) {
- threads[i].interrupt();
- try {
- threads[i].join();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- }
- return candidateMap;
- }
-
- private static StreamCandidate checkStream(
- final DistributedLogNamespace namespace,
- final String streamName,
- final OrderedScheduler scheduler) throws IOException {
- DistributedLogManager dlm = namespace.openLog(streamName);
- try {
- List<LogSegmentMetadata> segments = dlm.getLogSegments();
- if (segments.isEmpty()) {
- return null;
- }
- List<Future<LogSegmentCandidate>> futures =
- new ArrayList<Future<LogSegmentCandidate>>(segments.size());
- for (LogSegmentMetadata segment : segments) {
- futures.add(checkLogSegment(namespace, streamName, segment, scheduler));
- }
- List<LogSegmentCandidate> segmentCandidates;
- try {
- segmentCandidates = Await.result(Future.collect(futures));
- } catch (Exception e) {
- throw new IOException("Failed on checking stream " + streamName, e);
- }
- StreamCandidate streamCandidate = new StreamCandidate(streamName);
- for (LogSegmentCandidate segmentCandidate: segmentCandidates) {
- if (null != segmentCandidate) {
- streamCandidate.addLogSegmentCandidate(segmentCandidate);
- }
- }
- if (streamCandidate.segmentCandidates.isEmpty()) {
- return null;
- }
- return streamCandidate;
- } finally {
- dlm.close();
- }
- }
-
- private static Future<LogSegmentCandidate> checkLogSegment(
- final DistributedLogNamespace namespace,
- final String streamName,
- final LogSegmentMetadata metadata,
- final OrderedScheduler scheduler) {
- if (metadata.isInProgress()) {
- return Future.value(null);
- }
-
- final LogSegmentEntryStore entryStore = namespace.getNamespaceDriver()
- .getLogSegmentEntryStore(NamespaceDriver.Role.READER);
- return ReadUtils.asyncReadLastRecord(
- streamName,
- metadata,
- true,
- false,
- true,
- 4,
- 16,
- new AtomicInteger(0),
- scheduler,
- entryStore
- ).map(new Function<LogRecordWithDLSN, LogSegmentCandidate>() {
- @Override
- public LogSegmentCandidate apply(LogRecordWithDLSN record) {
- if (null != record &&
- (record.getDlsn().compareTo(metadata.getLastDLSN()) > 0 ||
- record.getTransactionId() > metadata.getLastTxId() ||
- !metadata.isRecordPositionWithinSegmentScope(record))) {
- return new LogSegmentCandidate(metadata, record);
- } else {
- return null;
- }
- }
- });
- }
-
- private static boolean repairStream(MetadataUpdater metadataUpdater,
- StreamCandidate streamCandidate,
- boolean verbose,
- boolean interactive) throws IOException {
- if (verbose) {
- System.out.println("Stream " + streamCandidate.streamName + " : ");
- for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
- System.out.println(" " + segmentCandidate.metadata.getLogSegmentSequenceNumber()
- + " : metadata = " + segmentCandidate.metadata + ", last dlsn = "
- + segmentCandidate.lastRecord.getDlsn());
- }
- System.out.println("-------------------------------------------");
- }
- if (interactive && !IOUtils.confirmPrompt("Do you want to fix the stream " + streamCandidate.streamName + " (Y/N) : ")) {
- return false;
- }
- for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
- LogSegmentMetadata newMetadata = FutureUtils.result(
- metadataUpdater.updateLastRecord(segmentCandidate.metadata, segmentCandidate.lastRecord));
- if (verbose) {
- System.out.println(" Fixed segment " + segmentCandidate.metadata.getLogSegmentSequenceNumber() + " : ");
- System.out.println(" old metadata : " + segmentCandidate.metadata);
- System.out.println(" new metadata : " + newMetadata);
- }
- }
- if (verbose) {
- System.out.println("-------------------------------------------");
- }
- return true;
- }
-
- //
- // Commands
- //
-
- /**
- * Unbind the bookkeeper environment for a given distributedlog uri.
- *
- * TODO: move unbind operation to namespace driver
- */
- class UnbindCommand extends OptsCommand {
-
- Options options = new Options();
-
- UnbindCommand() {
- super("unbind", "unbind the bookkeeper environment bound for a given distributedlog instance.");
- options.addOption("f", "force", false, "Force unbinding without prompt.");
- }
-
- @Override
- protected Options getOptions() {
- return options;
- }
-
- @Override
- protected String getUsage() {
- return "unbind [options] <distributedlog uri>";
- }
-
- @Override
- protected int runCmd(CommandLine cmdline) throws Exception {
- String[] args = cmdline.getArgs();
- if (args.length <= 0) {
- System.err.println("No distributedlog uri specified.");
- printUsage();
- return -1;
- }
- boolean force = cmdline.hasOption("f");
- URI uri = URI.create(args[0]);
- // resolving the uri to see if there is another bindings in this uri.
- ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri)
- .sessionTimeoutMs(10000).build();
- BKDLConfig bkdlConfig;
- try {
- bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
- } catch (IOException ie) {
- bkdlConfig = null;
- }
- if (null == bkdlConfig) {
- System.out.println("No bookkeeper is bound to " + uri);
- return 0;
- } else {
- System.out.println("There is bookkeeper bound to " + uri + " : ");
- System.out.println("");
- System.out.println(bkdlConfig.toString());
- System.out.println("");
- if (!force && !IOUtils.confirmPrompt("Do you want to unbind " + uri + " :\n")) {
- return 0;
- }
- }
- DLMetadata.unbind(uri);
- System.out.println("Unbound on " + uri + ".");
- return 0;
- }
- }
-
- /**
- * Bind Command to bind bookkeeper environment for a given distributed uri.
- *
- * TODO: move bind to namespace driver
- */
- class BindCommand extends OptsCommand {
-
- Options options = new Options();
-
- BindCommand() {
- super("bind", "bind the bookkeeper environment settings for a given distributedlog instance.");
- options.addOption("l", "bkLedgers", true, "ZooKeeper ledgers path for bookkeeper instance.");
- options.addOption("s", "bkZkServers", true, "ZooKeeper servers used for bookkeeper for writers.");
- options.addOption("bkzr", "bkZkServersForReader", true, "ZooKeeper servers used for bookkeeper for readers.");
- options.addOption("dlzw", "dlZkServersForWriter", true, "ZooKeeper servers used for distributedlog for writers.");
- options.addOption("dlzr", "dlZkServersForReader", true, "ZooKeeper servers used for distributedlog for readers.");
- options.addOption("i", "sanityCheckTxnID", true, "Flag to sanity check highest txn id.");
- options.addOption("r", "encodeRegionID", true, "Flag to encode region id.");
- options.addOption("seqno", "firstLogSegmentSeqNo", true, "The first log segment sequence number to use after upgrade");
- options.addOption("fns", "federatedNamespace", false, "Flag to turn a namespace to federated namespace");
- options.addOption("f", "force", false, "Force binding without prompt.");
- options.addOption("c", "creation", false, "Whether is it a creation binding.");
- options.addOption("q", "query", false, "Query the bookkeeper bindings");
- }
-
- @Override
- protected Options getOptions() {
- return options;
- }
-
- @Override
- protected String getUsage() {
- return "bind [options] <distributedlog uri>";
- }
-
- @Override
- protected int runCmd(CommandLine cmdline) throws Exception {
- boolean isQuery = cmdline.hasOption("q");
- if (!isQuery && (!cmdline.hasOption("l") || !cmdline.hasOption("s"))) {
- System.err.println("Error: Neither zkServers nor ledgersPath specified for bookkeeper environment.");
- printUsage();
- return -1;
- }
- String[] args = cmdline.getArgs();
- if (args.length <= 0) {
- System.err.println("No distributedlog uri specified.");
- printUsage();
- return -1;
- }
- boolean force = cmdline.hasOption("f");
- boolean creation = cmdline.hasOption("c");
- String bkLedgersPath = cmdline.getOptionValue("l");
- String bkZkServersForWriter = cmdline.getOptionValue("s");
- boolean sanityCheckTxnID =
- !cmdline.hasOption("i") || Boolean.parseBoolean(cmdline.getOptionValue("i"));
- boolean encodeRegionID =
- cmdline.hasOption("r") && Boolean.parseBoolean(cmdline.getOptionValue("r"));
-
- String bkZkServersForReader;
- if (cmdline.hasOption("bkzr")) {
- bkZkServersForReader = cmdline.getOptionValue("bkzr");
- } else {
- bkZkServersForReader = bkZkServersForWriter;
- }
-
- URI uri = URI.create(args[0]);
-
- String dlZkServersForWriter;
- String dlZkServersForReader;
- if (cmdline.hasOption("dlzw")) {
- dlZkServersForWriter = cmdline.getOptionValue("dlzw");
- } else {
- dlZkServersForWriter = BKNamespaceDriver.getZKServersFromDLUri(uri);
- }
- if (cmdline.hasOption("dlzr")) {
- dlZkServersForReader = cmdline.getOptionValue("dlzr");
- } else {
- dlZkServersForReader = dlZkServersForWriter;
- }
-
- // resolving the uri to see if there is another bindings in this uri.
- ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri).zkAclId(null)
- .sessionTimeoutMs(10000).build();
- try {
- BKDLConfig newBKDLConfig =
- new BKDLConfig(dlZkServersForWriter, dlZkServersForReader,
- bkZkServersForWriter, bkZkServersForReader, bkLedgersPath)
- .setSanityCheckTxnID(sanityCheckTxnID)
- .setEncodeRegionID(encodeRegionID);
-
- if (cmdline.hasOption("seqno")) {
- newBKDLConfig = newBKDLConfig.setFirstLogSegmentSeqNo(Long.parseLong(cmdline.getOptionValue("seqno")));
- }
-
- if (cmdline.hasOption("fns")) {
- newBKDLConfig = newBKDLConfig.setFederatedNamespace(true);
- }
-
- BKDLConfig bkdlConfig;
- try {
- bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
- } catch (IOException ie) {
- bkdlConfig = null;
- }
- if (null == bkdlConfig) {
- System.out.println("No bookkeeper is bound to " + uri);
- } else {
- System.out.println("There is bookkeeper bound to " + uri + " : ");
- System.out.println("");
- System.out.println(bkdlConfig.toString());
- System.out.println("");
- if (!isQuery) {
- if (newBKDLConfig.equals(bkdlConfig)) {
- System.out.println("No bookkeeper binding needs to be updated. Quit.");
- return 0;
- } else if(!newBKDLConfig.isFederatedNamespace() && bkdlConfig.isFederatedNamespace()) {
- System.out.println("You can't turn a federated namespace back to non-federated.");
- return 0;
- } else {
- if (!force && !IOUtils.confirmPrompt("Do you want to bind " + uri
- + " with new bookkeeper instance :\n" + newBKDLConfig)) {
- return 0;
- }
- }
- }
- }
- if (isQuery) {
- System.out.println("Done.");
- return 0;
- }
- DLMetadata dlMetadata = DLMetadata.create(newBKDLConfig);
- if (creation) {
- try {
- dlMetadata.create(uri);
- System.out.println("Created binding on " + uri + ".");
- } catch (IOException ie) {
- System.err.println("Failed to create binding on " + uri + " : " + ie.getMessage());
- }
- } else {
- try {
- dlMetadata.update(uri);
- System.out.println("Updated binding on " + uri + " : ");
- System.out.println("");
- System.out.println(newBKDLConfig.toString());
- System.out.println("");
- } catch (IOException ie) {
- System.err.println("Failed to update binding on " + uri + " : " + ie.getMessage());
- }
- }
- if (newBKDLConfig.isFederatedNamespace()) {
- try {
- FederatedZKLogMetadataStore.createFederatedNamespace(uri, zkc);
- } catch (KeeperException.NodeExistsException nee) {
- // ignore node exists exception
- }
- }
- return 0;
- } finally {
- zkc.close();
- }
- }
- }
-
- static class RepairSeqNoCommand extends PerDLCommand {
-
- boolean dryrun = false;
- boolean verbose = false;
- final List<String> streams = new ArrayList<String>();
-
- RepairSeqNoCommand() {
- super("repairseqno", "Repair a stream whose inprogress log segment has lower sequence number.");
- options.addOption("d", "dryrun", false, "Dry run without repairing");
- options.addOption("l", "list", true, "List of streams to repair, separated by comma");
- options.addOption("v", "verbose", false, "Print verbose messages");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- dryrun = cmdline.hasOption("d");
- verbose = cmdline.hasOption("v");
- force = !dryrun && cmdline.hasOption("f");
- if (!cmdline.hasOption("l")) {
- throw new ParseException("No streams provided to repair");
- }
- String streamsList = cmdline.getOptionValue("l");
- Collections.addAll(streams, streamsList.split(","));
- }
-
- @Override
- protected int runCmd() throws Exception {
- MetadataUpdater metadataUpdater = dryrun ?
- new DryrunLogSegmentMetadataStoreUpdater(getConf(),
- getLogSegmentMetadataStore()) :
- LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
- getLogSegmentMetadataStore());
- System.out.println("List of streams : ");
- System.out.println(streams);
- if (!IOUtils.confirmPrompt("Do you want to repair all these streams (Y/N):")) {
- return -1;
- }
- for (String stream : streams) {
- fixInprogressSegmentWithLowerSequenceNumber(getNamespace(), metadataUpdater, stream, verbose, !getForce());
- }
- return 0;
- }
-
- @Override
- protected String getUsage() {
- return "repairseqno [options]";
- }
- }
-
- static class DLCKCommand extends PerDLCommand {
-
- boolean dryrun = false;
- boolean verbose = false;
- int concurrency = 1;
-
- DLCKCommand() {
- super("dlck", "Check and repair a distributedlog namespace");
- options.addOption("d", "dryrun", false, "Dry run without repairing");
- options.addOption("v", "verbose", false, "Print verbose messages");
- options.addOption("cy", "concurrency", true, "Concurrency on checking streams");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- dryrun = cmdline.hasOption("d");
- verbose = cmdline.hasOption("v");
- if (cmdline.hasOption("cy")) {
- try {
- concurrency = Integer.parseInt(cmdline.getOptionValue("cy"));
- } catch (NumberFormatException nfe) {
- throw new ParseException("Invalid concurrency value : " + cmdline.getOptionValue("cy"));
- }
- }
- }
-
- @Override
- protected int runCmd() throws Exception {
- MetadataUpdater metadataUpdater = dryrun ?
- new DryrunLogSegmentMetadataStoreUpdater(getConf(),
- getLogSegmentMetadataStore()) :
- LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
- getLogSegmentMetadataStore());
- OrderedScheduler scheduler = OrderedScheduler.newBuilder()
- .name("dlck-scheduler")
- .corePoolSize(Runtime.getRuntime().availableProcessors())
- .build();
- ExecutorService executorService = Executors.newCachedThreadPool();
- try {
- checkAndRepairDLNamespace(getUri(), getNamespace(), metadataUpdater, scheduler,
- verbose, !getForce(), concurrency);
- } finally {
- SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
- }
- return 0;
- }
-
- @Override
- protected String getUsage() {
- return "dlck [options]";
- }
- }
-
- static class DeleteStreamACLCommand extends PerDLCommand {
-
- String stream = null;
-
- DeleteStreamACLCommand() {
- super("delete_stream_acl", "Delete ACL for a given stream");
- options.addOption("s", "stream", true, "Stream to set ACL");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- if (!cmdline.hasOption("s")) {
- throw new ParseException("No stream to set ACL");
- }
- stream = cmdline.getOptionValue("s");
- }
-
- @Override
- protected int runCmd() throws Exception {
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(getZooKeeperClient(), getUri());
- if (null == bkdlConfig.getACLRootPath()) {
- // acl isn't enabled for this namespace.
- System.err.println("ACL isn't enabled for namespace " + getUri());
- return -1;
- }
- String zkPath = getUri() + "/" + bkdlConfig.getACLRootPath() + "/" + stream;
- ZKAccessControl.delete(getZooKeeperClient(), zkPath);
- return 0;
- }
-
- @Override
- protected String getUsage() {
- return null;
- }
- }
-
- static class SetStreamACLCommand extends SetACLCommand {
-
- String stream = null;
-
- SetStreamACLCommand() {
- super("set_stream_acl", "Set Default ACL for a given stream");
- options.addOption("s", "stream", true, "Stream to set ACL");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- if (!cmdline.hasOption("s")) {
- throw new ParseException("No stream to set ACL");
- }
- stream = cmdline.getOptionValue("s");
- }
-
- @Override
- protected String getZKPath(String zkRootPath) {
- return zkRootPath + "/" + stream;
- }
-
- @Override
- protected String getUsage() {
- return "set_stream_acl [options]";
- }
- }
-
- static class SetDefaultACLCommand extends SetACLCommand {
-
- SetDefaultACLCommand() {
- super("set_default_acl", "Set Default ACL for a namespace");
- }
-
- @Override
- protected String getZKPath(String zkRootPath) {
- return zkRootPath;
- }
-
- @Override
- protected String getUsage() {
- return "set_default_acl [options]";
- }
- }
-
- static abstract class SetACLCommand extends PerDLCommand {
-
- boolean denyWrite = false;
- boolean denyTruncate = false;
- boolean denyDelete = false;
- boolean denyAcquire = false;
- boolean denyRelease = false;
-
- protected SetACLCommand(String name, String description) {
- super(name, description);
- options.addOption("dw", "deny-write", false, "Deny write/bulkWrite requests");
- options.addOption("dt", "deny-truncate", false, "Deny truncate requests");
- options.addOption("dd", "deny-delete", false, "Deny delete requests");
- options.addOption("da", "deny-acquire", false, "Deny acquire requests");
- options.addOption("dr", "deny-release", false, "Deny release requests");
- }
-
- @Override
- protected void parseCommandLine(CommandLine cmdline) throws ParseException {
- super.parseCommandLine(cmdline);
- denyWrite = cmdline.hasOption("dw");
- denyTruncate = cmdline.hasOption("dt");
- denyDelete = cmdline.hasOption("dd");
- denyAcquire = cmdline.hasOption("da");
- denyRelease = cmdline.hasOption("dr");
- }
-
- protected abstract String getZKPath(String zkRootPath);
-
- protected ZKAccessControl getZKAccessControl(ZooKeeperClient zkc, String zkPath) throws Exception {
- ZKAccessControl accessControl;
- try {
- accessControl = Await.result(ZKAccessControl.read(zkc, zkPath, null));
- } catch (KeeperException.NoNodeException nne) {
- accessControl = new ZKAccessControl(new AccessControlEntry(), zkPath);
- }
- return accessControl;
- }
-
- protected void setZKAccessControl(ZooKeeperClient zkc, ZKAccessControl accessControl) throws Exception {
- String zkPath = accessControl.getZKPath();
- if (null == zkc.get().exists(zkPath, false)) {
- accessControl.create(zkc);
- } else {
- accessControl.update(zkc);
- }
- }
-
- @Override
- protected int runCmd() throws Exception {
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(getZooKeeperClient(), getUri());
- if (null == bkdlConfig.getACLRootPath()) {
- // acl isn't enabled for this namespace.
- System.err.println("ACL isn't enabled for namespace " + getUri());
- return -1;
- }
- String zkPath = getZKPath(getUri().getPath() + "/" + bkdlConfig.getACLRootPath());
- ZKAccessControl accessControl = getZKAccessControl(getZooKeeperClient(), zkPath);
- AccessControlEntry acl = accessControl.getAccessControlEntry();
- acl.setDenyWrite(denyWrite);
- acl.setDenyTruncate(denyTruncate);
- acl.setDenyDelete(denyDelete);
- acl.setDenyAcquire(denyAcquire);
- acl.setDenyRelease(denyRelease);
- setZKAccessControl(getZooKeeperClient(), accessControl);
- return 0;
- }
-
- }
-
- public DistributedLogAdmin() {
- super();
- commands.clear();
- addCommand(new HelpCommand());
- addCommand(new BindCommand());
- addCommand(new UnbindCommand());
- addCommand(new RepairSeqNoCommand());
- addCommand(new DLCKCommand());
- addCommand(new SetDefaultACLCommand());
- addCommand(new SetStreamACLCommand());
- addCommand(new DeleteStreamACLCommand());
- }
-
- @Override
- protected String getName() {
- return "dlog_admin";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/package-info.java
deleted file mode 100644
index a7d6adb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Admin Tools for DistributedLog
- */
-package com.twitter.distributedlog.admin;
[26/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
----------------------------------------------------------------------
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
deleted file mode 100644
index a081606..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
+++ /dev/null
@@ -1,630 +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.auditor;
-
-import com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.SettableFuture;
-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.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;
-import org.apache.bookkeeper.client.BookKeeperAccessor;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.meta.LedgerManager;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * DL Auditor will audit DL namespace, e.g. find leaked ledger, report disk usage by streams.
- */
-public class DLAuditor {
-
- private static final Logger logger = LoggerFactory.getLogger(DLAuditor.class);
-
- private final DistributedLogConfiguration conf;
-
- public DLAuditor(DistributedLogConfiguration conf) {
- this.conf = conf;
- }
-
- private ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) {
- NamespaceDriver driver = namespace.getNamespaceDriver();
- assert(driver instanceof BKNamespaceDriver);
- return ((BKNamespaceDriver) driver).getWriterZKC();
- }
-
- 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 = BKNamespaceDriver.getZKServersFromDLUri(firstURI);
- for (URI uri : uris) {
- if (!zkServers.equalsIgnoreCase(BKNamespaceDriver.getZKServersFromDLUri(uri))) {
- throw new IllegalArgumentException("Uris don't belong to same zookeeper cluster");
- }
- }
- return zkServers;
- }
-
- private BKDLConfig resolveBKDLConfig(ZooKeeperClient zkc, List<URI> uris) throws IOException {
- URI firstURI = uris.get(0);
- BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, firstURI);
- for (URI uri : uris) {
- BKDLConfig anotherConfig = BKDLConfig.resolveDLConfig(zkc, uri);
- if (!(Objects.equal(bkdlConfig.getBkLedgersPath(), anotherConfig.getBkLedgersPath())
- && Objects.equal(bkdlConfig.getBkZkServersForWriter(), anotherConfig.getBkZkServersForWriter()))) {
- throw new IllegalArgumentException("Uris don't use same bookkeeper cluster");
- }
- }
- return bkdlConfig;
- }
-
- public Pair<Set<Long>, Set<Long>> collectLedgers(List<URI> uris, List<List<String>> allocationPaths)
- throws IOException {
- Preconditions.checkArgument(uris.size() > 0, "No uri provided to audit");
-
- String zkServers = validateAndGetZKServers(uris);
- RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
- conf.getZKRetryBackoffStartMillis(),
- conf.getZKRetryBackoffMaxMillis(),
- Integer.MAX_VALUE);
- ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
- .name("DLAuditor-ZK")
- .zkServers(zkServers)
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .retryPolicy(retryPolicy)
- .zkAclId(conf.getZkAclId())
- .build();
- ExecutorService executorService = Executors.newCachedThreadPool();
- try {
- BKDLConfig bkdlConfig = resolveBKDLConfig(zkc, uris);
- logger.info("Resolved bookkeeper config : {}", bkdlConfig);
-
- BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
- .name("DLAuditor-BK")
- .dlConfig(conf)
- .zkServers(bkdlConfig.getBkZkServersForWriter())
- .ledgersPath(bkdlConfig.getBkLedgersPath())
- .build();
- try {
- Set<Long> bkLedgers = collectLedgersFromBK(bkc, executorService);
- Set<Long> dlLedgers = collectLedgersFromDL(uris, allocationPaths);
- return Pair.of(bkLedgers, dlLedgers);
- } finally {
- bkc.close();
- }
- } finally {
- zkc.close();
- executorService.shutdown();
- }
- }
-
- /**
- * Find leak ledgers phase 1: collect ledgers set.
- */
- private Set<Long> collectLedgersFromBK(BookKeeperClient bkc,
- final ExecutorService executorService)
- throws IOException {
- LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
-
- final Set<Long> ledgers = new HashSet<Long>();
- final SettableFuture<Void> doneFuture = SettableFuture.create();
-
- BookkeeperInternalCallbacks.Processor<Long> collector =
- new BookkeeperInternalCallbacks.Processor<Long>() {
- @Override
- public void process(Long lid,
- final AsyncCallback.VoidCallback cb) {
- synchronized (ledgers) {
- ledgers.add(lid);
- if (0 == ledgers.size() % 1000) {
- logger.info("Collected {} ledgers", ledgers.size());
- }
- }
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- cb.processResult(BKException.Code.OK, null, null);
- }
- });
-
- }
- };
- AsyncCallback.VoidCallback finalCb = new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx) {
- if (BKException.Code.OK == rc) {
- doneFuture.set(null);
- } else {
- doneFuture.setException(BKException.create(rc));
- }
- }
- };
- lm.asyncProcessLedgers(collector, finalCb, null, BKException.Code.OK,
- BKException.Code.ZKException);
- try {
- doneFuture.get();
- logger.info("Collected total {} ledgers", ledgers.size());
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new DLInterruptedException("Interrupted on collecting ledgers : ", e);
- } catch (ExecutionException e) {
- if (e.getCause() instanceof IOException) {
- throw (IOException)(e.getCause());
- } else {
- throw new IOException("Failed to collect ledgers : ", e.getCause());
- }
- }
- return ledgers;
- }
-
- /**
- * Find leak ledgers phase 2: collect ledgers from uris.
- */
- private Set<Long> collectLedgersFromDL(List<URI> uris, List<List<String>> allocationPaths)
- throws IOException {
- final Set<Long> ledgers = new TreeSet<Long>();
- List<DistributedLogNamespace> namespaces =
- new ArrayList<DistributedLogNamespace>(uris.size());
- try {
- for (URI uri : uris) {
- 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 (final DistributedLogNamespace namespace : namespaces) {
- final DistributedLogNamespace dlNamespace = namespace;
- final URI uri = uris.get(i);
- final List<String> aps = allocationPaths.get(i);
- i++;
- executor.submit(new Runnable() {
- @Override
- public void run() {
- try {
- logger.info("Collecting ledgers from {} : {}", uri, aps);
- collectLedgersFromAllocator(uri, namespace, aps, ledgers);
- synchronized (ledgers) {
- logger.info("Collected {} ledgers from allocators for {} : {} ",
- new Object[]{ledgers.size(), uri, ledgers});
- }
- collectLedgersFromDL(uri, namespace, ledgers);
- } catch (IOException e) {
- numFailures.incrementAndGet();
- logger.info("Error to collect ledgers from DL : ", e);
- }
- doneLatch.countDown();
- }
- });
- }
- try {
- doneLatch.await();
- if (numFailures.get() > 0) {
- throw new IOException(numFailures.get() + " errors to collect ledgers from DL");
- }
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- logger.warn("Interrupted on collecting ledgers from DL : ", e);
- throw new DLInterruptedException("Interrupted on collecting ledgers from DL : ", e);
- }
- } finally {
- executor.shutdown();
- }
- } finally {
- for (DistributedLogNamespace namespace : namespaces) {
- namespace.close();
- }
- }
- return ledgers;
- }
-
- private void collectLedgersFromAllocator(final URI uri,
- final DistributedLogNamespace namespace,
- final List<String> allocationPaths,
- final Set<Long> ledgers) throws IOException {
- final LinkedBlockingQueue<String> poolQueue =
- new LinkedBlockingQueue<String>();
- for (String allocationPath : allocationPaths) {
- String rootPath = uri.getPath() + "/" + allocationPath;
- try {
- List<String> pools = getZooKeeperClient(namespace).get().getChildren(rootPath, false);
- for (String pool : pools) {
- poolQueue.add(rootPath + "/" + pool);
- }
- } catch (KeeperException e) {
- throw new ZKException("Failed to get list of pools from " + rootPath, e);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new DLInterruptedException("Interrupted on getting list of pools from " + rootPath, e);
- }
- }
-
-
- logger.info("Collecting ledgers from allocators for {} : {}", uri, poolQueue);
-
- executeAction(poolQueue, 10, new Action<String>() {
- @Override
- public void execute(String poolPath) throws IOException {
- try {
- collectLedgersFromPool(poolPath);
- } catch (InterruptedException e) {
- throw new DLInterruptedException("Interrupted on collecting ledgers from allocation pool " + poolPath, e);
- } catch (KeeperException e) {
- throw new ZKException("Failed to collect ledgers from allocation pool " + poolPath, e.code());
- }
- }
-
- private void collectLedgersFromPool(String poolPath)
- throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
- List<String> allocators = getZooKeeperClient(namespace).get()
- .getChildren(poolPath, false);
- for (String allocator : allocators) {
- String allocatorPath = poolPath + "/" + allocator;
- byte[] data = getZooKeeperClient(namespace).get().getData(allocatorPath, false, new Stat());
- if (null != data && data.length > 0) {
- try {
- long ledgerId = DLUtils.bytes2LogSegmentId(data);
- synchronized (ledgers) {
- ledgers.add(ledgerId);
- }
- } catch (NumberFormatException nfe) {
- logger.warn("Invalid ledger found in allocator path {} : ", allocatorPath, nfe);
- }
- }
- }
- }
- });
-
- logger.info("Collected ledgers from allocators for {}.", uri);
- }
-
- private void collectLedgersFromDL(final URI uri,
- final DistributedLogNamespace namespace,
- final Set<Long> ledgers) throws IOException {
- logger.info("Enumerating {} to collect streams.", uri);
- Iterator<String> streams = namespace.getLogs();
- final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
- while (streams.hasNext()) {
- streamQueue.add(streams.next());
- }
-
- logger.info("Collected {} streams from uri {} : {}",
- new Object[] { streamQueue.size(), uri, streams });
-
- executeAction(streamQueue, 10, new Action<String>() {
- @Override
- public void execute(String stream) throws IOException {
- collectLedgersFromStream(namespace, stream, ledgers);
- }
- });
- }
-
- private List<Long> collectLedgersFromStream(DistributedLogNamespace namespace,
- String stream,
- Set<Long> ledgers)
- throws IOException {
- DistributedLogManager dlm = namespace.openLog(stream);
- try {
- List<LogSegmentMetadata> segments = dlm.getLogSegments();
- List<Long> sLedgers = new ArrayList<Long>();
- for (LogSegmentMetadata segment : segments) {
- synchronized (ledgers) {
- ledgers.add(segment.getLogSegmentId());
- }
- sLedgers.add(segment.getLogSegmentId());
- }
- return sLedgers;
- } finally {
- dlm.close();
- }
- }
-
- /**
- * Calculating stream space usage from given <i>uri</i>.
- *
- * @param uri dl uri
- * @throws IOException
- */
- public Map<String, Long> calculateStreamSpaceUsage(final URI uri) throws IOException {
- logger.info("Collecting stream space usage for {}.", uri);
- DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
- .conf(conf)
- .uri(uri)
- .build();
- try {
- return calculateStreamSpaceUsage(uri, namespace);
- } finally {
- namespace.close();
- }
- }
-
- private Map<String, Long> calculateStreamSpaceUsage(
- final URI uri, final DistributedLogNamespace namespace)
- throws IOException {
- Iterator<String> streams = namespace.getLogs();
- final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
- while (streams.hasNext()) {
- streamQueue.add(streams.next());
- }
-
- final Map<String, Long> streamSpaceUsageMap =
- new ConcurrentSkipListMap<String, Long>();
- final AtomicInteger numStreamsCollected = new AtomicInteger(0);
-
- executeAction(streamQueue, 10, new Action<String>() {
- @Override
- public void execute(String stream) throws IOException {
- streamSpaceUsageMap.put(stream,
- calculateStreamSpaceUsage(namespace, stream));
- if (numStreamsCollected.incrementAndGet() % 1000 == 0) {
- logger.info("Calculated {} streams from uri {}.", numStreamsCollected.get(), uri);
- }
- }
- });
-
- return streamSpaceUsageMap;
- }
-
- private long calculateStreamSpaceUsage(final DistributedLogNamespace namespace,
- final String stream) throws IOException {
- DistributedLogManager dlm = namespace.openLog(stream);
- long totalBytes = 0;
- try {
- List<LogSegmentMetadata> segments = dlm.getLogSegments();
- for (LogSegmentMetadata segment : segments) {
- try {
- LedgerHandle lh = getBookKeeperClient(namespace).get().openLedgerNoRecovery(segment.getLogSegmentId(),
- BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
- totalBytes += lh.getLength();
- lh.close();
- } catch (BKException e) {
- logger.error("Failed to open ledger {} : ", segment.getLogSegmentId(), e);
- throw new IOException("Failed to open ledger " + segment.getLogSegmentId(), e);
- } catch (InterruptedException e) {
- logger.warn("Interrupted on opening ledger {} : ", segment.getLogSegmentId(), e);
- Thread.currentThread().interrupt();
- throw new DLInterruptedException("Interrupted on opening ledger " + segment.getLogSegmentId(), e);
- }
- }
- } finally {
- dlm.close();
- }
- return totalBytes;
- }
-
- public long calculateLedgerSpaceUsage(URI uri) throws IOException {
- List<URI> uris = Lists.newArrayList(uri);
- String zkServers = validateAndGetZKServers(uris);
- RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
- conf.getZKRetryBackoffStartMillis(),
- conf.getZKRetryBackoffMaxMillis(),
- Integer.MAX_VALUE);
- ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
- .name("DLAuditor-ZK")
- .zkServers(zkServers)
- .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
- .retryPolicy(retryPolicy)
- .zkAclId(conf.getZkAclId())
- .build();
- ExecutorService executorService = Executors.newCachedThreadPool();
- try {
- BKDLConfig bkdlConfig = resolveBKDLConfig(zkc, uris);
- logger.info("Resolved bookkeeper config : {}", bkdlConfig);
-
- BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
- .name("DLAuditor-BK")
- .dlConfig(conf)
- .zkServers(bkdlConfig.getBkZkServersForWriter())
- .ledgersPath(bkdlConfig.getBkLedgersPath())
- .build();
- try {
- return calculateLedgerSpaceUsage(bkc, executorService);
- } finally {
- bkc.close();
- }
- } finally {
- zkc.close();
- executorService.shutdown();
- }
- }
-
- private long calculateLedgerSpaceUsage(BookKeeperClient bkc,
- final ExecutorService executorService)
- throws IOException {
- final AtomicLong totalBytes = new AtomicLong(0);
- final AtomicLong totalEntries = new AtomicLong(0);
- final AtomicLong numLedgers = new AtomicLong(0);
-
- LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
-
- final SettableFuture<Void> doneFuture = SettableFuture.create();
- final BookKeeper bk = bkc.get();
-
- BookkeeperInternalCallbacks.Processor<Long> collector =
- new BookkeeperInternalCallbacks.Processor<Long>() {
- @Override
- public void process(final Long lid,
- final AsyncCallback.VoidCallback cb) {
- numLedgers.incrementAndGet();
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- bk.asyncOpenLedgerNoRecovery(lid, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8),
- new org.apache.bookkeeper.client.AsyncCallback.OpenCallback() {
- @Override
- public void openComplete(int rc, LedgerHandle lh, Object ctx) {
- final int cbRc;
- if (BKException.Code.OK == rc) {
- totalBytes.addAndGet(lh.getLength());
- totalEntries.addAndGet(lh.getLastAddConfirmed() + 1);
- cbRc = rc;
- } else {
- cbRc = BKException.Code.ZKException;
- }
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- cb.processResult(cbRc, null, null);
- }
- });
- }
- }, null);
- }
- });
- }
- };
- AsyncCallback.VoidCallback finalCb = new AsyncCallback.VoidCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx) {
- if (BKException.Code.OK == rc) {
- doneFuture.set(null);
- } else {
- doneFuture.setException(BKException.create(rc));
- }
- }
- };
- lm.asyncProcessLedgers(collector, finalCb, null, BKException.Code.OK, BKException.Code.ZKException);
- try {
- doneFuture.get();
- logger.info("calculated {} ledgers\n\ttotal bytes = {}\n\ttotal entries = {}",
- new Object[] { numLedgers.get(), totalBytes.get(), totalEntries.get() });
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw new DLInterruptedException("Interrupted on calculating ledger space : ", e);
- } catch (ExecutionException e) {
- if (e.getCause() instanceof IOException) {
- throw (IOException)(e.getCause());
- } else {
- throw new IOException("Failed to calculate ledger space : ", e.getCause());
- }
- }
- return totalBytes.get();
- }
-
- public void close() {
- // no-op
- }
-
- static interface Action<T> {
- void execute(T item) throws IOException ;
- }
-
- static <T> void executeAction(final LinkedBlockingQueue<T> queue,
- final int numThreads,
- final Action<T> action) throws IOException {
- final CountDownLatch failureLatch = new CountDownLatch(1);
- final CountDownLatch doneLatch = new CountDownLatch(queue.size());
- final AtomicInteger numFailures = new AtomicInteger(0);
- final AtomicInteger completedThreads = new AtomicInteger(0);
-
- ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
- try {
- for (int i = 0 ; i < numThreads; i++) {
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- while (true) {
- T item = queue.poll();
- if (null == item) {
- break;
- }
- try {
- action.execute(item);
- } catch (IOException ioe) {
- logger.error("Failed to execute action on item '{}'", item, ioe);
- numFailures.incrementAndGet();
- failureLatch.countDown();
- break;
- }
- doneLatch.countDown();
- }
- if (numFailures.get() == 0 && completedThreads.incrementAndGet() == numThreads) {
- failureLatch.countDown();
- }
- }
- });
- }
- try {
- failureLatch.await();
- if (numFailures.get() > 0) {
- throw new IOException("Encountered " + numFailures.get() + " failures on executing action.");
- }
- doneLatch.await();
- } catch (InterruptedException ie) {
- Thread.currentThread().interrupt();
- logger.warn("Interrupted on executing action", ie);
- throw new DLInterruptedException("Interrupted on executing action", ie);
- }
- } finally {
- executorService.shutdown();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/DynamicQuorumConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/DynamicQuorumConfigProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/DynamicQuorumConfigProvider.java
deleted file mode 100644
index 871997f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/DynamicQuorumConfigProvider.java
+++ /dev/null
@@ -1,37 +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.bk;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-
-/**
- * Provider returns quorum configs based on dynamic configuration.
- */
-public class DynamicQuorumConfigProvider implements QuorumConfigProvider {
-
- private final DynamicDistributedLogConfiguration conf;
-
- public DynamicQuorumConfigProvider(DynamicDistributedLogConfiguration conf) {
- this.conf = conf;
- }
-
- @Override
- public QuorumConfig getQuorumConfig() {
- return conf.getQuorumConfig();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/ImmutableQuorumConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/ImmutableQuorumConfigProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/ImmutableQuorumConfigProvider.java
deleted file mode 100644
index 6c3f06e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/ImmutableQuorumConfigProvider.java
+++ /dev/null
@@ -1,35 +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.bk;
-
-/**
- * Provider that returns an immutable quorum config.
- */
-public class ImmutableQuorumConfigProvider implements QuorumConfigProvider {
-
- private final QuorumConfig quorumConfig;
-
- public ImmutableQuorumConfigProvider(QuorumConfig quorumConfig) {
- this.quorumConfig = quorumConfig;
- }
-
- @Override
- public QuorumConfig getQuorumConfig() {
- return quorumConfig;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocator.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocator.java
deleted file mode 100644
index c14f374..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocator.java
+++ /dev/null
@@ -1,32 +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.bk;
-
-import com.twitter.distributedlog.util.Allocator;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-import java.io.IOException;
-
-public interface LedgerAllocator extends Allocator<LedgerHandle, Object> {
-
- /**
- * Start the ledger allocator. The implementaion should not be blocking call.
- */
- void start() throws IOException;
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorDelegator.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorDelegator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorDelegator.java
deleted file mode 100644
index b76d03a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorDelegator.java
+++ /dev/null
@@ -1,83 +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.bk;
-
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-import java.io.IOException;
-
-/**
- * Delegator of the underlying allocator. If it owns the allocator, it takes
- * the responsibility of start the allocator and close the allocator.
- */
-public class LedgerAllocatorDelegator implements LedgerAllocator {
-
- private final LedgerAllocator allocator;
- private final boolean ownAllocator;
-
- /**
- * Create an allocator's delegator.
- *
- * @param allocator
- * the underlying allocator
- * @param ownAllocator
- * whether to own the allocator
- */
- public LedgerAllocatorDelegator(LedgerAllocator allocator,
- boolean ownAllocator)
- throws IOException {
- this.allocator = allocator;
- this.ownAllocator = ownAllocator;
- if (this.ownAllocator) {
- this.allocator.start();
- }
- }
-
- @Override
- public void start() throws IOException {
- // no-op
- }
-
- @Override
- public Future<Void> delete() {
- return Future.exception(new UnsupportedOperationException("Can't delete an allocator by delegator"));
- }
-
- @Override
- public void allocate() throws IOException {
- this.allocator.allocate();
- }
-
- @Override
- public Future<LedgerHandle> tryObtain(Transaction<Object> txn,
- OpListener<LedgerHandle> listener) {
- return this.allocator.tryObtain(txn, listener);
- }
-
- @Override
- public Future<Void> asyncClose() {
- if (ownAllocator) {
- return this.allocator.asyncClose();
- } else {
- return Future.value(null);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorPool.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorPool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorPool.java
deleted file mode 100644
index dd0894e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorPool.java
+++ /dev/null
@@ -1,458 +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.bk;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Function;
-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.meta.ZkVersion;
-import org.apache.bookkeeper.util.ZkUtils;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class LedgerAllocatorPool implements LedgerAllocator {
-
- static final Logger logger = LoggerFactory.getLogger(LedgerAllocatorPool.class);
-
- private final DistributedLogConfiguration conf;
- private final QuorumConfigProvider quorumConfigProvider;
- private final BookKeeperClient bkc;
- private final ZooKeeperClient zkc;
- private final ScheduledExecutorService scheduledExecutorService;
- private final String poolPath;
- private final int corePoolSize;
-
- private final LinkedList<SimpleLedgerAllocator> pendingList =
- new LinkedList<SimpleLedgerAllocator>();
- private final LinkedList<SimpleLedgerAllocator> allocatingList =
- new LinkedList<SimpleLedgerAllocator>();
- private final Map<String, SimpleLedgerAllocator> rescueMap =
- new HashMap<String, SimpleLedgerAllocator>();
- private final Map<LedgerHandle, SimpleLedgerAllocator> obtainMap =
- new HashMap<LedgerHandle, SimpleLedgerAllocator>();
- private final Map<SimpleLedgerAllocator, LedgerHandle> reverseObtainMap =
- new HashMap<SimpleLedgerAllocator, LedgerHandle>();
-
- public LedgerAllocatorPool(String poolPath, int corePoolSize,
- DistributedLogConfiguration conf,
- ZooKeeperClient zkc,
- BookKeeperClient bkc,
- ScheduledExecutorService scheduledExecutorService) throws IOException {
- this.poolPath = poolPath;
- this.corePoolSize = corePoolSize;
- this.conf = conf;
- this.quorumConfigProvider =
- new ImmutableQuorumConfigProvider(conf.getQuorumConfig());
- this.zkc = zkc;
- this.bkc = bkc;
- this.scheduledExecutorService = scheduledExecutorService;
- initializePool();
- }
-
- @Override
- public void start() throws IOException {
- for (LedgerAllocator allocator : pendingList) {
- // issue allocating requests during initialize
- allocator.allocate();
- }
- }
-
- @VisibleForTesting
- synchronized int pendingListSize() {
- return pendingList.size();
- }
-
- @VisibleForTesting
- synchronized int allocatingListSize() {
- return allocatingList.size();
- }
-
- @VisibleForTesting
- public synchronized int obtainMapSize() {
- return obtainMap.size();
- }
-
- @VisibleForTesting
- synchronized int rescueSize() {
- return rescueMap.size();
- }
-
- @VisibleForTesting
- synchronized SimpleLedgerAllocator getLedgerAllocator(LedgerHandle lh) {
- return obtainMap.get(lh);
- }
-
- private void initializePool() throws IOException {
- try {
- List<String> allocators;
- try {
- allocators = zkc.get().getChildren(poolPath, false);
- } catch (KeeperException.NoNodeException e) {
- logger.info("Allocator Pool {} doesn't exist. Creating it.", poolPath);
- ZkUtils.createFullPathOptimistic(zkc.get(), poolPath, new byte[0], zkc.getDefaultACL(),
- CreateMode.PERSISTENT);
- allocators = zkc.get().getChildren(poolPath, false);
- }
- if (null == allocators) {
- allocators = new ArrayList<String>();
- }
- if (allocators.size() < corePoolSize) {
- createAllocators(corePoolSize - allocators.size());
- allocators = zkc.get().getChildren(poolPath, false);
- }
- initializeAllocators(allocators);
- } catch (InterruptedException ie) {
- throw new DLInterruptedException("Interrupted when ensuring " + poolPath + " created : ", ie);
- } catch (KeeperException ke) {
- throw new IOException("Encountered zookeeper exception when initializing pool " + poolPath + " : ", ke);
- }
- }
-
- private void createAllocators(int numAllocators) throws InterruptedException, IOException {
- final AtomicInteger numPendings = new AtomicInteger(numAllocators);
- final AtomicInteger numFailures = new AtomicInteger(0);
- final CountDownLatch latch = new CountDownLatch(1);
- AsyncCallback.StringCallback createCallback = new AsyncCallback.StringCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, String name) {
- if (KeeperException.Code.OK.intValue() != rc) {
- numFailures.incrementAndGet();
- latch.countDown();
- return;
- }
- if (numPendings.decrementAndGet() == 0 && numFailures.get() == 0) {
- latch.countDown();
- }
- }
- };
- for (int i = 0; i < numAllocators; i++) {
- zkc.get().create(poolPath + "/A", new byte[0],
- zkc.getDefaultACL(),
- CreateMode.PERSISTENT_SEQUENTIAL,
- createCallback, null);
- }
- latch.await();
- if (numFailures.get() > 0) {
- throw new IOException("Failed to create " + numAllocators + " allocators.");
- }
- }
-
- /**
- * Initialize simple allocators with given list of allocator names <i>allocators</i>.
- * It initializes a simple allocator with its simple allocator path.
- */
- private void initializeAllocators(List<String> allocators) throws IOException, InterruptedException {
- final AtomicInteger numPendings = new AtomicInteger(allocators.size());
- final AtomicInteger numFailures = new AtomicInteger(0);
- final CountDownLatch latch = new CountDownLatch(numPendings.get() > 0 ? 1 : 0);
- AsyncCallback.DataCallback dataCallback = new AsyncCallback.DataCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
- if (KeeperException.Code.OK.intValue() != rc) {
- numFailures.incrementAndGet();
- latch.countDown();
- return;
- }
- Versioned<byte[]> allocatorData =
- new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
- SimpleLedgerAllocator allocator =
- new SimpleLedgerAllocator(path, allocatorData, quorumConfigProvider, zkc, bkc);
- allocator.start();
- pendingList.add(allocator);
- if (numPendings.decrementAndGet() == 0 && numFailures.get() == 0) {
- latch.countDown();
- }
- }
- };
- for (String name : allocators) {
- String path = poolPath + "/" + name;
- zkc.get().getData(path, false, dataCallback, null);
- }
- latch.await();
- if (numFailures.get() > 0) {
- throw new IOException("Failed to initialize allocators : " + allocators);
- }
- }
-
- private void scheduleAllocatorRescue(final SimpleLedgerAllocator ledgerAllocator) {
- try {
- scheduledExecutorService.schedule(new Runnable() {
- @Override
- public void run() {
- try {
- rescueAllocator(ledgerAllocator);
- } catch (DLInterruptedException dle) {
- Thread.currentThread().interrupt();
- }
- }
- }, conf.getZKRetryBackoffStartMillis(), TimeUnit.MILLISECONDS);
- } catch (RejectedExecutionException ree) {
- logger.warn("Failed to schedule rescuing ledger allocator {} : ", ledgerAllocator.allocatePath, ree);
- }
- }
-
- /**
- * Rescue a ledger allocator from an ERROR state
- * @param ledgerAllocator
- * ledger allocator to rescue
- */
- private void rescueAllocator(final SimpleLedgerAllocator ledgerAllocator) throws DLInterruptedException {
- SimpleLedgerAllocator oldAllocator;
- synchronized (this) {
- oldAllocator = rescueMap.put(ledgerAllocator.allocatePath, ledgerAllocator);
- }
- if (oldAllocator != null) {
- logger.info("ledger allocator {} is being rescued.", ledgerAllocator.allocatePath);
- return;
- }
- try {
- zkc.get().getData(ledgerAllocator.allocatePath, false, new AsyncCallback.DataCallback() {
- @Override
- public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
- boolean retry = false;
- SimpleLedgerAllocator newAllocator = null;
- if (KeeperException.Code.OK.intValue() == rc) {
- Versioned<byte[]> allocatorData =
- new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
- logger.info("Rescuing ledger allocator {}.", path);
- newAllocator = new SimpleLedgerAllocator(path, allocatorData, quorumConfigProvider, zkc, bkc);
- newAllocator.start();
- logger.info("Rescued ledger allocator {}.", path);
- } else if (KeeperException.Code.NONODE.intValue() == rc) {
- logger.info("Ledger allocator {} doesn't exist, skip rescuing it.", path);
- } else {
- retry = true;
- }
- synchronized (LedgerAllocatorPool.this) {
- rescueMap.remove(ledgerAllocator.allocatePath);
- if (null != newAllocator) {
- pendingList.addLast(newAllocator);
- }
- }
- if (retry) {
- scheduleAllocatorRescue(ledgerAllocator);
- }
- }
- }, null);
- } catch (InterruptedException ie) {
- logger.warn("Interrupted on rescuing ledger allocator {} : ", ledgerAllocator.allocatePath, ie);
- synchronized (LedgerAllocatorPool.this) {
- rescueMap.remove(ledgerAllocator.allocatePath);
- }
- throw new DLInterruptedException("Interrupted on rescuing ledger allocator " + ledgerAllocator.allocatePath, ie);
- } catch (IOException ioe) {
- logger.warn("Failed to rescue ledger allocator {}, retry rescuing it later : ", ledgerAllocator.allocatePath, ioe);
- synchronized (LedgerAllocatorPool.this) {
- rescueMap.remove(ledgerAllocator.allocatePath);
- }
- scheduleAllocatorRescue(ledgerAllocator);
- }
- }
-
- @Override
- public void allocate() throws IOException {
- SimpleLedgerAllocator allocator;
- synchronized (this) {
- if (pendingList.isEmpty()) {
- // if no ledger allocator available, we should fail it immediately, which the request will be redirected to other
- // proxies
- throw new IOException("No ledger allocator available under " + poolPath + ".");
- } else {
- allocator = pendingList.removeFirst();
- }
- }
- boolean success = false;
- try {
- allocator.allocate();
- synchronized (this) {
- allocatingList.addLast(allocator);
- }
- success = true;
- } finally {
- if (!success) {
- rescueAllocator(allocator);
- }
- }
- }
-
- @Override
- public Future<LedgerHandle> tryObtain(final Transaction<Object> txn,
- final Transaction.OpListener<LedgerHandle> listener) {
- final SimpleLedgerAllocator allocator;
- synchronized (this) {
- if (allocatingList.isEmpty()) {
- return Future.exception(new IOException("No ledger allocator available under " + poolPath + "."));
- } else {
- allocator = allocatingList.removeFirst();
- }
- }
-
- final Promise<LedgerHandle> tryObtainPromise = new Promise<LedgerHandle>();
- final FutureEventListener<LedgerHandle> tryObtainListener = new FutureEventListener<LedgerHandle>() {
- @Override
- public void onSuccess(LedgerHandle lh) {
- synchronized (LedgerAllocatorPool.this) {
- obtainMap.put(lh, allocator);
- reverseObtainMap.put(allocator, lh);
- tryObtainPromise.setValue(lh);
- }
- }
-
- @Override
- public void onFailure(Throwable cause) {
- try {
- rescueAllocator(allocator);
- } catch (IOException ioe) {
- logger.info("Failed to rescue allocator {}", allocator.allocatePath, ioe);
- }
- tryObtainPromise.setException(cause);
- }
- };
-
- allocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
- @Override
- public void onCommit(LedgerHandle lh) {
- confirmObtain(allocator);
- listener.onCommit(lh);
- }
-
- @Override
- public void onAbort(Throwable t) {
- abortObtain(allocator);
- listener.onAbort(t);
- }
- }).addEventListener(tryObtainListener);
- return tryObtainPromise;
- }
-
- void confirmObtain(SimpleLedgerAllocator allocator) {
- synchronized (this) {
- LedgerHandle lh = reverseObtainMap.remove(allocator);
- if (null != lh) {
- obtainMap.remove(lh);
- }
- }
- synchronized (this) {
- pendingList.addLast(allocator);
- }
- }
-
- void abortObtain(SimpleLedgerAllocator allocator) {
- synchronized (this) {
- LedgerHandle lh = reverseObtainMap.remove(allocator);
- if (null != lh) {
- obtainMap.remove(lh);
- }
- }
- // if a ledger allocator is aborted, it is better to rescue it. since the ledger allocator might
- // already encounter BadVersion exception.
- try {
- rescueAllocator(allocator);
- } catch (DLInterruptedException e) {
- logger.warn("Interrupted on rescuing ledger allocator pool {} : ", poolPath, e);
- Thread.currentThread().interrupt();
- }
- }
-
- @Override
- public Future<Void> asyncClose() {
- List<LedgerAllocator> allocatorsToClose;
- synchronized (this) {
- allocatorsToClose = Lists.newArrayListWithExpectedSize(
- pendingList.size() + allocatingList.size() + obtainMap.size());
- for (LedgerAllocator allocator : pendingList) {
- allocatorsToClose.add(allocator);
- }
- for (LedgerAllocator allocator : allocatingList) {
- allocatorsToClose.add(allocator);
- }
- for (LedgerAllocator allocator : obtainMap.values()) {
- allocatorsToClose.add(allocator);
- }
- }
- return FutureUtils.processList(allocatorsToClose, new Function<LedgerAllocator, Future<Void>>() {
- @Override
- public Future<Void> apply(LedgerAllocator allocator) {
- return allocator.asyncClose();
- }
- }, scheduledExecutorService).map(new AbstractFunction1<List<Void>, Void>() {
- @Override
- public Void apply(List<Void> values) {
- return null;
- }
- });
- }
-
- @Override
- public Future<Void> delete() {
- List<LedgerAllocator> allocatorsToDelete;
- synchronized (this) {
- allocatorsToDelete = Lists.newArrayListWithExpectedSize(
- pendingList.size() + allocatingList.size() + obtainMap.size());
- for (LedgerAllocator allocator : pendingList) {
- allocatorsToDelete.add(allocator);
- }
- for (LedgerAllocator allocator : allocatingList) {
- allocatorsToDelete.add(allocator);
- }
- for (LedgerAllocator allocator : obtainMap.values()) {
- allocatorsToDelete.add(allocator);
- }
- }
- return FutureUtils.processList(allocatorsToDelete, new Function<LedgerAllocator, Future<Void>>() {
- @Override
- public Future<Void> apply(LedgerAllocator allocator) {
- return allocator.delete();
- }
- }, scheduledExecutorService).flatMap(new AbstractFunction1<List<Void>, Future<Void>>() {
- @Override
- public Future<Void> apply(List<Void> values) {
- return Utils.zkDelete(zkc, poolPath, new ZkVersion(-1));
- }
- });
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorUtils.java
deleted file mode 100644
index 0db6d74..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorUtils.java
+++ /dev/null
@@ -1,54 +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.bk;
-
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-
-import java.io.IOException;
-import java.util.concurrent.ScheduledExecutorService;
-
-public class LedgerAllocatorUtils {
-
- /**
- * Create ledger allocator pool.
- *
- * @param poolPath
- * ledger allocator pool path.
- * @param corePoolSize
- * ledger allocator pool core size.
- * @param conf
- * distributedlog configuration.
- * @param zkc
- * zookeeper client
- * @param bkc
- * bookkeeper client
- * @return ledger allocator
- * @throws IOException
- */
- public static LedgerAllocator createLedgerAllocatorPool(
- String poolPath,
- int corePoolSize,
- DistributedLogConfiguration conf,
- ZooKeeperClient zkc,
- BookKeeperClient bkc,
- ScheduledExecutorService scheduledExecutorService) throws IOException {
- return new LedgerAllocatorPool(poolPath, corePoolSize, conf, zkc, bkc, scheduledExecutorService);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfig.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfig.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfig.java
deleted file mode 100644
index a9cc16c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfig.java
+++ /dev/null
@@ -1,92 +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.bk;
-
-import com.google.common.base.Objects;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Configuration for quorums
- */
-public class QuorumConfig {
-
- private static final Logger logger = LoggerFactory.getLogger(QuorumConfig.class);
-
- private final int ensembleSize;
- private final int writeQuorumSize;
- private final int ackQuorumSize;
-
- public QuorumConfig(int ensembleSize,
- int writeQuorumSize,
- int ackQuorumSize) {
- this.ensembleSize = ensembleSize;
- if (this.ensembleSize < writeQuorumSize) {
- this.writeQuorumSize = this.ensembleSize;
- logger.warn("Setting write quorum size {} greater than ensemble size {}",
- writeQuorumSize, this.ensembleSize);
- } else {
- this.writeQuorumSize = writeQuorumSize;
- }
- if (this.writeQuorumSize < ackQuorumSize) {
- this.ackQuorumSize = this.writeQuorumSize;
- logger.warn("Setting write ack quorum size {} greater than write quorum size {}",
- ackQuorumSize, this.writeQuorumSize);
- } else {
- this.ackQuorumSize = ackQuorumSize;
- }
- }
-
- public int getEnsembleSize() {
- return ensembleSize;
- }
-
- public int getWriteQuorumSize() {
- return writeQuorumSize;
- }
-
- public int getAckQuorumSize() {
- return ackQuorumSize;
- }
-
- @Override
- public int hashCode() {
- return Objects.hashCode(ensembleSize, writeQuorumSize, ackQuorumSize);
- }
-
- @Override
- public boolean equals(Object obj) {
- if (!(obj instanceof QuorumConfig)) {
- return false;
- }
- QuorumConfig other = (QuorumConfig) obj;
- return ensembleSize == other.ensembleSize
- && writeQuorumSize == other.writeQuorumSize
- && ackQuorumSize == other.ackQuorumSize;
- }
-
- @Override
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("QuorumConfig[ensemble=")
- .append(ensembleSize).append(", write quorum=")
- .append(writeQuorumSize).append(", ack quorum=")
- .append(ackQuorumSize).append("]");
- return sb.toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfigProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfigProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfigProvider.java
deleted file mode 100644
index 2f65427..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfigProvider.java
+++ /dev/null
@@ -1,32 +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.bk;
-
-/**
- * Provider to provide quorum config
- */
-public interface QuorumConfigProvider {
-
- /**
- * Get the quorum config for a given log stream.
- *
- * @return quorum config
- */
- QuorumConfig getQuorumConfig();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/SimpleLedgerAllocator.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/SimpleLedgerAllocator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/SimpleLedgerAllocator.java
deleted file mode 100644
index ab5976e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/SimpleLedgerAllocator.java
+++ /dev/null
@@ -1,536 +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.bk;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.zk.ZKTransaction;
-import com.twitter.distributedlog.zk.ZKVersionedSetOp;
-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.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * Allocator to allocate ledgers.
- */
-public class SimpleLedgerAllocator implements LedgerAllocator, FutureEventListener<LedgerHandle>, OpListener<Version> {
-
- static final Logger LOG = LoggerFactory.getLogger(SimpleLedgerAllocator.class);
-
- static enum Phase {
- ALLOCATING, ALLOCATED, HANDING_OVER, HANDED_OVER, ERROR
- }
-
- static class AllocationException extends IOException {
-
- private static final long serialVersionUID = -1111397872059426882L;
-
- private final Phase phase;
-
- public AllocationException(Phase phase, String msg) {
- super(msg);
- this.phase = phase;
- }
-
- public Phase getPhase() {
- return this.phase;
- }
-
- }
-
- static class ConcurrentObtainException extends AllocationException {
-
- private static final long serialVersionUID = -8532471098537176913L;
-
- public ConcurrentObtainException(Phase phase, String msg) {
- super(phase, msg);
- }
- }
-
- // zookeeper client
- final ZooKeeperClient zkc;
- // bookkeeper client
- final BookKeeperClient bkc;
- // znode path
- final String allocatePath;
- // allocation phase
- Phase phase = Phase.HANDED_OVER;
- // version
- ZkVersion version = new ZkVersion(-1);
- // outstanding allocation
- Promise<LedgerHandle> allocatePromise;
- // outstanding tryObtain transaction
- Transaction<Object> tryObtainTxn = null;
- OpListener<LedgerHandle> tryObtainListener = null;
- // ledger id left from previous allocation
- Long ledgerIdLeftFromPrevAllocation = null;
- // Allocated Ledger
- LedgerHandle allocatedLh = null;
-
- Future<Void> closeFuture = null;
- final LinkedList<Future<Void>> ledgerDeletions =
- new LinkedList<Future<Void>>();
-
- // Ledger configuration
- private final QuorumConfigProvider quorumConfigProvider;
-
- static Future<Versioned<byte[]>> getAndCreateAllocationData(final String allocatePath,
- final ZooKeeperClient zkc) {
- return Utils.zkGetData(zkc, allocatePath, false)
- .flatMap(new AbstractFunction1<Versioned<byte[]>, Future<Versioned<byte[]>>>() {
- @Override
- public Future<Versioned<byte[]>> apply(Versioned<byte[]> result) {
- if (null != result && null != result.getVersion() && null != result.getValue()) {
- return Future.value(result);
- }
- return createAllocationData(allocatePath, zkc);
- }
- });
- }
-
- private static Future<Versioned<byte[]>> createAllocationData(final String allocatePath,
- final ZooKeeperClient zkc) {
- try {
- final Promise<Versioned<byte[]>> promise = new Promise<Versioned<byte[]>>();
- zkc.get().create(allocatePath, DistributedLogConstants.EMPTY_BYTES,
- zkc.getDefaultACL(), CreateMode.PERSISTENT,
- new org.apache.zookeeper.AsyncCallback.Create2Callback() {
- @Override
- public void processResult(int rc, String path, Object ctx, String name, Stat stat) {
- if (KeeperException.Code.OK.intValue() == rc) {
- promise.setValue(new Versioned<byte[]>(DistributedLogConstants.EMPTY_BYTES,
- new ZkVersion(stat.getVersion())));
- } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
- Utils.zkGetData(zkc, allocatePath, false).proxyTo(promise);
- } else {
- promise.setException(FutureUtils.zkException(
- KeeperException.create(KeeperException.Code.get(rc)), allocatePath));
- }
- }
- }, null);
- return promise;
- } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
- return Future.exception(FutureUtils.zkException(e, allocatePath));
- } catch (InterruptedException e) {
- return Future.exception(FutureUtils.zkException(e, allocatePath));
- }
- }
-
- public static Future<SimpleLedgerAllocator> of(final String allocatePath,
- final Versioned<byte[]> allocationData,
- final QuorumConfigProvider quorumConfigProvider,
- final ZooKeeperClient zkc,
- final BookKeeperClient bkc) {
- if (null != allocationData && null != allocationData.getValue()
- && null != allocationData.getVersion()) {
- return Future.value(new SimpleLedgerAllocator(allocatePath, allocationData,
- quorumConfigProvider, zkc, bkc));
- }
- return getAndCreateAllocationData(allocatePath, zkc)
- .map(new AbstractFunction1<Versioned<byte[]>, SimpleLedgerAllocator>() {
- @Override
- public SimpleLedgerAllocator apply(Versioned<byte[]> allocationData) {
- return new SimpleLedgerAllocator(allocatePath, allocationData,
- quorumConfigProvider, zkc, bkc);
- }
- });
- }
-
- /**
- * Construct a ledger allocator.
- *
- * @param allocatePath
- * znode path to store the allocated ledger.
- * @param allocationData
- * allocation data.
- * @param quorumConfigProvider
- * Quorum configuration provider.
- * @param zkc
- * zookeeper client.
- * @param bkc
- * bookkeeper client.
- */
- public SimpleLedgerAllocator(String allocatePath,
- Versioned<byte[]> allocationData,
- QuorumConfigProvider quorumConfigProvider,
- ZooKeeperClient zkc,
- BookKeeperClient bkc) {
- this.zkc = zkc;
- this.bkc = bkc;
- this.allocatePath = allocatePath;
- this.quorumConfigProvider = quorumConfigProvider;
- initialize(allocationData);
- }
-
- /**
- * Initialize the allocator.
- *
- * @param allocationData
- * Allocation Data.
- */
- private void initialize(Versioned<byte[]> allocationData) {
- setVersion((ZkVersion) allocationData.getVersion());
- byte[] data = allocationData.getValue();
- if (null != data && data.length > 0) {
- // delete the allocated ledger since this is left by last allocation.
- try {
- ledgerIdLeftFromPrevAllocation = DLUtils.bytes2LogSegmentId(data);
- } catch (NumberFormatException nfe) {
- LOG.warn("Invalid data found in allocator path {} : ", allocatePath, nfe);
- }
- }
-
- }
-
- private synchronized void deleteLedgerLeftFromPreviousAllocationIfNecessary() {
- if (null != ledgerIdLeftFromPrevAllocation) {
- LOG.info("Deleting allocated-but-unused ledger left from previous allocation {}.", ledgerIdLeftFromPrevAllocation);
- deleteLedger(ledgerIdLeftFromPrevAllocation);
- ledgerIdLeftFromPrevAllocation = null;
- }
- }
-
- @Override
- public synchronized void allocate() throws IOException {
- if (Phase.ERROR == phase) {
- throw new AllocationException(Phase.ERROR, "Error on ledger allocator for " + allocatePath);
- }
- if (Phase.HANDED_OVER == phase) {
- // issue an allocate request when ledger is already handed over.
- allocateLedger();
- }
- }
-
- @Override
- public synchronized Future<LedgerHandle> tryObtain(final Transaction<Object> txn,
- final OpListener<LedgerHandle> listener) {
- if (Phase.ERROR == phase) {
- return Future.exception(new AllocationException(Phase.ERROR,
- "Error on allocating ledger under " + allocatePath));
- }
- if (Phase.HANDING_OVER == phase || Phase.HANDED_OVER == phase || null != tryObtainTxn) {
- return Future.exception(new ConcurrentObtainException(phase,
- "Ledger handle is handling over to another thread : " + phase));
- }
- tryObtainTxn = txn;
- tryObtainListener = listener;
- if (null != allocatedLh) {
- completeAllocation(allocatedLh);
- }
- return allocatePromise;
- }
-
- @Override
- public void onCommit(Version r) {
- confirmObtain((ZkVersion) r);
- }
-
- private void confirmObtain(ZkVersion zkVersion) {
- boolean shouldAllocate = false;
- OpListener<LedgerHandle> listenerToNotify = null;
- LedgerHandle lhToNotify = null;
- synchronized (this) {
- if (Phase.HANDING_OVER == phase) {
- setPhase(Phase.HANDED_OVER);
- setVersion(zkVersion);
- listenerToNotify = tryObtainListener;
- lhToNotify = allocatedLh;
- // reset the state
- allocatedLh = null;
- allocatePromise = null;
- tryObtainTxn = null;
- tryObtainListener = null;
- // mark flag to issue an allocation request
- shouldAllocate = true;
- }
- }
- if (null != listenerToNotify && null != lhToNotify) {
- // notify the listener
- listenerToNotify.onCommit(lhToNotify);
- }
- if (shouldAllocate) {
- // issue an allocation request
- allocateLedger();
- }
- }
-
- @Override
- public void onAbort(Throwable t) {
- OpListener<LedgerHandle> listenerToNotify;
- synchronized (this) {
- listenerToNotify = tryObtainListener;
- if (t instanceof KeeperException &&
- ((KeeperException) t).code() == KeeperException.Code.BADVERSION) {
- LOG.info("Set ledger allocator {} to ERROR state after hit bad version : version = {}",
- allocatePath, getVersion());
- setPhase(Phase.ERROR);
- } else {
- if (Phase.HANDING_OVER == phase) {
- setPhase(Phase.ALLOCATED);
- tryObtainTxn = null;
- tryObtainListener = null;
- }
- }
- }
- if (null != listenerToNotify) {
- listenerToNotify.onAbort(t);
- }
- }
-
- private synchronized void setPhase(Phase phase) {
- this.phase = phase;
- LOG.info("Ledger allocator {} moved to phase {} : version = {}.",
- new Object[] { allocatePath, phase, version });
- }
-
- private synchronized void allocateLedger() {
- // make sure previous allocation is already handed over.
- if (Phase.HANDED_OVER != phase) {
- LOG.error("Trying allocate ledger for {} in phase {}, giving up.", allocatePath, phase);
- return;
- }
- setPhase(Phase.ALLOCATING);
- allocatePromise = new Promise<LedgerHandle>();
- QuorumConfig quorumConfig = quorumConfigProvider.getQuorumConfig();
- bkc.createLedger(
- quorumConfig.getEnsembleSize(),
- quorumConfig.getWriteQuorumSize(),
- quorumConfig.getAckQuorumSize()
- ).addEventListener(this);
- }
-
- private synchronized void completeAllocation(LedgerHandle lh) {
- allocatedLh = lh;
- if (null == tryObtainTxn) {
- return;
- }
- org.apache.zookeeper.Op zkSetDataOp = org.apache.zookeeper.Op.setData(
- allocatePath, DistributedLogConstants.EMPTY_BYTES, version.getZnodeVersion());
- ZKVersionedSetOp commitOp = new ZKVersionedSetOp(zkSetDataOp, this);
- tryObtainTxn.addOp(commitOp);
- setPhase(Phase.HANDING_OVER);
- FutureUtils.setValue(allocatePromise, lh);
- }
-
- private synchronized void failAllocation(Throwable cause) {
- FutureUtils.setException(allocatePromise, cause);
- }
-
- @Override
- public void onSuccess(LedgerHandle lh) {
- // a ledger is created, update the ledger to allocation path before handling it over for usage.
- markAsAllocated(lh);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- LOG.error("Error creating ledger for allocating {} : ", allocatePath, cause);
- setPhase(Phase.ERROR);
- failAllocation(cause);
- }
-
- private synchronized ZkVersion getVersion() {
- return version;
- }
-
- private synchronized void setVersion(ZkVersion newVersion) {
- Version.Occurred occurred = newVersion.compare(version);
- if (occurred == Version.Occurred.AFTER) {
- LOG.info("Ledger allocator for {} moved version from {} to {}.",
- new Object[] { allocatePath, version, newVersion });
- version = newVersion;
- } else {
- LOG.warn("Ledger allocator for {} received an old version {}, current version is {}.",
- new Object[] { allocatePath, newVersion , version });
- }
- }
-
- private void markAsAllocated(final LedgerHandle lh) {
- byte[] data = DLUtils.logSegmentId2Bytes(lh.getId());
- Utils.zkSetData(zkc, allocatePath, data, getVersion())
- .addEventListener(new FutureEventListener<ZkVersion>() {
- @Override
- public void onSuccess(ZkVersion version) {
- // we only issue deleting ledger left from previous allocation when we could allocate first ledger
- // as zookeeper version could prevent us doing stupid things.
- deleteLedgerLeftFromPreviousAllocationIfNecessary();
- setVersion(version);
- setPhase(Phase.ALLOCATED);
- // complete the allocation after it is marked as allocated
- completeAllocation(lh);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- setPhase(Phase.ERROR);
- deleteLedger(lh.getId());
- LOG.error("Fail mark ledger {} as allocated under {} : ",
- new Object[] { lh.getId(), allocatePath, cause });
- // fail the allocation since failed to mark it as allocated
- failAllocation(cause);
- }
- });
- }
-
- void deleteLedger(final long ledgerId) {
- final Future<Void> deleteFuture = bkc.deleteLedger(ledgerId, true);
- synchronized (ledgerDeletions) {
- ledgerDeletions.add(deleteFuture);
- }
- deleteFuture.onFailure(new AbstractFunction1<Throwable, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Throwable cause) {
- LOG.error("Error deleting ledger {} for ledger allocator {}, retrying : ",
- new Object[] { ledgerId, allocatePath, cause });
- if (!isClosing()) {
- deleteLedger(ledgerId);
- }
- return BoxedUnit.UNIT;
- }
- }).ensure(new AbstractFunction0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- synchronized (ledgerDeletions) {
- ledgerDeletions.remove(deleteFuture);
- }
- return BoxedUnit.UNIT;
- }
- });
- }
-
- private synchronized boolean isClosing() {
- return closeFuture != null;
- }
-
- private Future<Void> closeInternal(boolean cleanup) {
- Promise<Void> closePromise;
- synchronized (this) {
- if (null != closeFuture) {
- return closeFuture;
- }
- closePromise = new Promise<Void>();
- closeFuture = closePromise;
- }
- if (!cleanup) {
- LOG.info("Abort ledger allocator without cleaning up on {}.", allocatePath);
- FutureUtils.setValue(closePromise, null);
- return closePromise;
- }
- cleanupAndClose(closePromise);
- return closePromise;
- }
-
- private void cleanupAndClose(final Promise<Void> closePromise) {
- LOG.info("Closing ledger allocator on {}.", allocatePath);
- final ZKTransaction txn = new ZKTransaction(zkc);
- // try obtain ledger handle
- tryObtain(txn, new OpListener<LedgerHandle>() {
- @Override
- public void onCommit(LedgerHandle r) {
- // no-op
- complete();
- }
-
- @Override
- public void onAbort(Throwable t) {
- // no-op
- complete();
- }
-
- private void complete() {
- FutureUtils.setValue(closePromise, null);
- LOG.info("Closed ledger allocator on {}.", allocatePath);
- }
- }).addEventListener(new FutureEventListener<LedgerHandle>() {
- @Override
- public void onSuccess(LedgerHandle lh) {
- // try obtain succeed
- // if we could obtain the ledger handle, we have the responsibility to close it
- deleteLedger(lh.getId());
- // wait for deletion to be completed
- List<Future<Void>> outstandingDeletions;
- synchronized (ledgerDeletions) {
- outstandingDeletions = Lists.newArrayList(ledgerDeletions);
- }
- Future.collect(outstandingDeletions).addEventListener(new FutureEventListener<List<Void>>() {
- @Override
- public void onSuccess(List<Void> values) {
- txn.execute();
- }
-
- @Override
- public void onFailure(Throwable cause) {
- LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : ", cause);
- FutureUtils.setValue(closePromise, null);
- }
- });
- }
-
- @Override
- public void onFailure(Throwable cause) {
- LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : ", cause);
- FutureUtils.setValue(closePromise, null);
- }
- });
-
- }
-
- @Override
- public void start() {
- // nop
- }
-
- @Override
- public Future<Void> asyncClose() {
- return closeInternal(false);
- }
-
- @Override
- public Future<Void> delete() {
- return closeInternal(true).flatMap(new AbstractFunction1<Void, Future<Void>>() {
- @Override
- public Future<Void> apply(Void value) {
- return Utils.zkDelete(zkc, allocatePath, getVersion());
- }
- });
- }
-
-}
[44/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClientBuilder.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClientBuilder.java
deleted file mode 100644
index 3f65aff..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClientBuilder.java
+++ /dev/null
@@ -1,608 +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.service;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.DistributedLogClientImpl;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.client.proxy.ClusterClient;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.distributedlog.client.routing.RegionsRoutingService;
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.client.routing.RoutingUtils;
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.finagle.Name;
-import com.twitter.finagle.Resolver$;
-import com.twitter.finagle.Service;
-import com.twitter.finagle.ThriftMux;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId;
-import com.twitter.finagle.thrift.ThriftClientFramedCodec;
-import com.twitter.finagle.thrift.ThriftClientRequest;
-import com.twitter.util.Duration;
-import java.net.SocketAddress;
-import java.net.URI;
-import java.util.Random;
-import org.apache.commons.lang.StringUtils;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Option;
-
-/**
- * Builder to build {@link DistributedLogClient}.
- */
-public final class DistributedLogClientBuilder {
-
- private static final Logger logger = LoggerFactory.getLogger(DistributedLogClientBuilder.class);
-
- private static final Random random = new Random(System.currentTimeMillis());
-
- private String name = null;
- private ClientId clientId = null;
- private RoutingService.Builder routingServiceBuilder = null;
- private ClientBuilder clientBuilder = null;
- private String serverRoutingServiceFinagleName = null;
- private StatsReceiver statsReceiver = new NullStatsReceiver();
- private StatsReceiver streamStatsReceiver = new NullStatsReceiver();
- private ClientConfig clientConfig = new ClientConfig();
- private boolean enableRegionStats = false;
- private final RegionResolver regionResolver = new DefaultRegionResolver();
-
- /**
- * Create a client builder.
- *
- * @return client builder
- */
- public static DistributedLogClientBuilder newBuilder() {
- return new DistributedLogClientBuilder();
- }
-
- /**
- * Create a new client builder from an existing {@code builder}.
- *
- * @param builder the existing builder.
- * @return a new client builder.
- */
- public static DistributedLogClientBuilder newBuilder(DistributedLogClientBuilder builder) {
- DistributedLogClientBuilder newBuilder = new DistributedLogClientBuilder();
- newBuilder.name = builder.name;
- newBuilder.clientId = builder.clientId;
- newBuilder.clientBuilder = builder.clientBuilder;
- newBuilder.routingServiceBuilder = builder.routingServiceBuilder;
- newBuilder.statsReceiver = builder.statsReceiver;
- newBuilder.streamStatsReceiver = builder.streamStatsReceiver;
- newBuilder.enableRegionStats = builder.enableRegionStats;
- newBuilder.serverRoutingServiceFinagleName = builder.serverRoutingServiceFinagleName;
- newBuilder.clientConfig = ClientConfig.newConfig(builder.clientConfig);
- return newBuilder;
- }
-
- // private constructor
- private DistributedLogClientBuilder() {}
-
- /**
- * Client Name.
- *
- * @param name
- * client name
- * @return client builder.
- */
- public DistributedLogClientBuilder name(String name) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.name = name;
- return newBuilder;
- }
-
- /**
- * Client ID.
- *
- * @param clientId
- * client id
- * @return client builder.
- */
- public DistributedLogClientBuilder clientId(ClientId clientId) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientId = clientId;
- return newBuilder;
- }
-
- /**
- * Serverset to access proxy services.
- *
- * @param serverSet
- * server set.
- * @return client builder.
- */
- public DistributedLogClientBuilder serverSet(ServerSet serverSet) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(serverSet);
- newBuilder.enableRegionStats = false;
- return newBuilder;
- }
-
- /**
- * Server Sets to access proxy services.
- *
- * <p>The <i>local</i> server set will be tried first then <i>remotes</i>.
- *
- * @param local local server set.
- * @param remotes remote server sets.
- * @return client builder.
- */
- public DistributedLogClientBuilder serverSets(ServerSet local, ServerSet...remotes) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- RoutingService.Builder[] builders = new RoutingService.Builder[remotes.length + 1];
- builders[0] = RoutingUtils.buildRoutingService(local);
- for (int i = 1; i < builders.length; i++) {
- builders[i] = RoutingUtils.buildRoutingService(remotes[i - 1]);
- }
- newBuilder.routingServiceBuilder = RegionsRoutingService.newBuilder()
- .resolver(regionResolver)
- .routingServiceBuilders(builders);
- newBuilder.enableRegionStats = remotes.length > 0;
- return newBuilder;
- }
-
- /**
- * Name to access proxy services.
- *
- * @param finagleNameStr
- * finagle name string.
- * @return client builder.
- */
- public DistributedLogClientBuilder finagleNameStr(String finagleNameStr) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(finagleNameStr);
- newBuilder.enableRegionStats = false;
- return newBuilder;
- }
-
- /**
- * Finagle name strs to access proxy services.
- *
- * <p>The <i>local</i> finalge name str will be tried first, then <i>remotes</i>.
- *
- * @param local local server set.
- * @param remotes remote server sets.
- * @return client builder.
- */
- public DistributedLogClientBuilder finagleNameStrs(String local, String...remotes) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- RoutingService.Builder[] builders = new RoutingService.Builder[remotes.length + 1];
- builders[0] = RoutingUtils.buildRoutingService(local);
- for (int i = 1; i < builders.length; i++) {
- builders[i] = RoutingUtils.buildRoutingService(remotes[i - 1]);
- }
- newBuilder.routingServiceBuilder = RegionsRoutingService.newBuilder()
- .routingServiceBuilders(builders)
- .resolver(regionResolver);
- newBuilder.enableRegionStats = remotes.length > 0;
- return newBuilder;
- }
-
- /**
- * URI to access proxy services.
- *
- * <p>Assuming the write proxies are announced under `.write_proxy` of the provided namespace uri.
- * The builder will convert the dl uri (e.g. distributedlog://{zkserver}/path/to/namespace) to
- * zookeeper serverset based finagle name str (`zk!{zkserver}!/path/to/namespace/.write_proxy`)
- *
- * @param uri namespace uri to access the serverset of write proxies
- * @return distributedlog builder
- */
- public DistributedLogClientBuilder uri(URI uri) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- String zkServers = uri.getAuthority().replace(";", ",");
- String[] zkServerList = StringUtils.split(zkServers, ',');
- String finagleNameStr = String.format(
- "zk!%s!%s/.write_proxy",
- zkServerList[random.nextInt(zkServerList.length)], // zk server
- uri.getPath());
- newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(finagleNameStr);
- newBuilder.enableRegionStats = false;
- return newBuilder;
- }
-
- /**
- * Address of write proxy to connect.
- *
- * @param address
- * write proxy address.
- * @return client builder.
- */
- public DistributedLogClientBuilder host(SocketAddress address) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(address);
- newBuilder.enableRegionStats = false;
- return newBuilder;
- }
-
- private DistributedLogClientBuilder routingServiceBuilder(RoutingService.Builder builder) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.routingServiceBuilder = builder;
- newBuilder.enableRegionStats = false;
- return newBuilder;
- }
-
- /**
- * Routing Service to access proxy services.
- *
- * @param routingService
- * routing service
- * @return client builder.
- */
- @VisibleForTesting
- public DistributedLogClientBuilder routingService(RoutingService routingService) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(routingService);
- newBuilder.enableRegionStats = false;
- return newBuilder;
- }
-
- /**
- * Stats receiver to expose client stats.
- *
- * @param statsReceiver
- * stats receiver.
- * @return client builder.
- */
- public DistributedLogClientBuilder statsReceiver(StatsReceiver statsReceiver) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.statsReceiver = statsReceiver;
- return newBuilder;
- }
-
- /**
- * Stream Stats Receiver to expose per stream stats.
- *
- * @param streamStatsReceiver
- * stream stats receiver
- * @return client builder.
- */
- public DistributedLogClientBuilder streamStatsReceiver(StatsReceiver streamStatsReceiver) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.streamStatsReceiver = streamStatsReceiver;
- return newBuilder;
- }
-
- /**
- * Set underlying finagle client builder.
- *
- * @param builder
- * finagle client builder.
- * @return client builder.
- */
- public DistributedLogClientBuilder clientBuilder(ClientBuilder builder) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientBuilder = builder;
- return newBuilder;
- }
-
- /**
- * Backoff time when redirecting to an already retried host.
- *
- * @param ms
- * backoff time.
- * @return client builder.
- */
- public DistributedLogClientBuilder redirectBackoffStartMs(int ms) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setRedirectBackoffStartMs(ms);
- return newBuilder;
- }
-
- /**
- * Max backoff time when redirecting to an already retried host.
- *
- * @param ms
- * backoff time.
- * @return client builder.
- */
- public DistributedLogClientBuilder redirectBackoffMaxMs(int ms) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setRedirectBackoffMaxMs(ms);
- return newBuilder;
- }
-
- /**
- * Max redirects that is allowed per request.
- *
- * <p>If <i>redirects</i> are exhausted, fail the request immediately.
- *
- * @param redirects
- * max redirects allowed before failing a request.
- * @return client builder.
- */
- public DistributedLogClientBuilder maxRedirects(int redirects) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setMaxRedirects(redirects);
- return newBuilder;
- }
-
- /**
- * Timeout per request in millis.
- *
- * @param timeoutMs
- * timeout per request in millis.
- * @return client builder.
- */
- public DistributedLogClientBuilder requestTimeoutMs(int timeoutMs) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setRequestTimeoutMs(timeoutMs);
- return newBuilder;
- }
-
- /**
- * Set thriftmux enabled.
- *
- * @param enabled
- * is thriftmux enabled
- * @return client builder.
- */
- public DistributedLogClientBuilder thriftmux(boolean enabled) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setThriftMux(enabled);
- return newBuilder;
- }
-
- /**
- * Set failfast stream exception handling enabled.
- *
- * @param enabled
- * is failfast exception handling enabled
- * @return client builder.
- */
- public DistributedLogClientBuilder streamFailfast(boolean enabled) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setStreamFailfast(enabled);
- return newBuilder;
- }
-
- /**
- * Set the regex to match stream names that the client cares about.
- *
- * @param nameRegex
- * stream name regex
- * @return client builder
- */
- public DistributedLogClientBuilder streamNameRegex(String nameRegex) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setStreamNameRegex(nameRegex);
- return newBuilder;
- }
-
- /**
- * Whether to use the new handshake endpoint to exchange ownership cache.
- *
- * <p>Enable this when the servers are updated to support handshaking with client info.
- *
- * @param enabled
- * new handshake endpoint is enabled.
- * @return client builder.
- */
- public DistributedLogClientBuilder handshakeWithClientInfo(boolean enabled) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setHandshakeWithClientInfo(enabled);
- return newBuilder;
- }
-
- /**
- * Set the periodic handshake interval in milliseconds.
- *
- * <p>Every <code>intervalMs</code>, the DL client will handshake with existing proxies again.
- * If the interval is less than ownership sync interval, the handshake won't sync ownerships. Otherwise, it will.
- *
- * @see #periodicOwnershipSyncIntervalMs(long)
- * @param intervalMs
- * handshake interval
- * @return client builder.
- */
- public DistributedLogClientBuilder periodicHandshakeIntervalMs(long intervalMs) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setPeriodicHandshakeIntervalMs(intervalMs);
- return newBuilder;
- }
-
- /**
- * Set the periodic ownership sync interval in milliseconds.
- *
- * <p>If periodic handshake is enabled, the handshake will sync ownership if the elapsed time is larger than
- * sync interval.
- *
- * @see #periodicHandshakeIntervalMs(long)
- * @param intervalMs
- * interval that handshake should sync ownerships.
- * @return client builder
- */
- public DistributedLogClientBuilder periodicOwnershipSyncIntervalMs(long intervalMs) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setPeriodicOwnershipSyncIntervalMs(intervalMs);
- return newBuilder;
- }
-
- /**
- * Enable/Disable periodic dumping ownership cache.
- *
- * @param enabled
- * flag to enable/disable periodic dumping ownership cache
- * @return client builder.
- */
- public DistributedLogClientBuilder periodicDumpOwnershipCache(boolean enabled) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setPeriodicDumpOwnershipCacheEnabled(enabled);
- return newBuilder;
- }
-
- /**
- * Set periodic dumping ownership cache interval.
- *
- * @param intervalMs
- * interval on dumping ownership cache, in millis.
- * @return client builder
- */
- public DistributedLogClientBuilder periodicDumpOwnershipCacheIntervalMs(long intervalMs) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setPeriodicDumpOwnershipCacheIntervalMs(intervalMs);
- return newBuilder;
- }
-
- /**
- * Enable handshake tracing.
- *
- * @param enabled
- * flag to enable/disable handshake tracing
- * @return client builder
- */
- public DistributedLogClientBuilder handshakeTracing(boolean enabled) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setHandshakeTracingEnabled(enabled);
- return newBuilder;
- }
-
- /**
- * Enable checksum on requests to the proxy.
- *
- * @param enabled
- * flag to enable/disable checksum
- * @return client builder
- */
- public DistributedLogClientBuilder checksum(boolean enabled) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig.setChecksumEnabled(enabled);
- return newBuilder;
- }
-
- /**
- * Configure the finagle name string for the server-side routing service.
- *
- * @param nameStr name string of the server-side routing service
- * @return client builder
- */
- public DistributedLogClientBuilder serverRoutingServiceFinagleNameStr(String nameStr) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.serverRoutingServiceFinagleName = nameStr;
- return newBuilder;
- }
-
- DistributedLogClientBuilder clientConfig(ClientConfig clientConfig) {
- DistributedLogClientBuilder newBuilder = newBuilder(this);
- newBuilder.clientConfig = ClientConfig.newConfig(clientConfig);
- return newBuilder;
- }
-
- /**
- * Build distributedlog client.
- *
- * @return distributedlog client.
- */
- public DistributedLogClient build() {
- return buildClient();
- }
-
- /**
- * Build monitor service client.
- *
- * @return monitor service client.
- */
- public MonitorServiceClient buildMonitorClient() {
-
- return buildClient();
- }
-
- @SuppressWarnings("unchecked")
- ClusterClient buildServerRoutingServiceClient(String serverRoutingServiceFinagleName) {
- ClientBuilder builder = this.clientBuilder;
- if (null == builder) {
- builder = ClientBuilder.get()
- .tcpConnectTimeout(Duration.fromMilliseconds(200))
- .connectTimeout(Duration.fromMilliseconds(200))
- .requestTimeout(Duration.fromSeconds(1))
- .retries(20);
- if (!clientConfig.getThriftMux()) {
- builder = builder.hostConnectionLimit(1);
- }
- }
- if (clientConfig.getThriftMux()) {
- builder = builder.stack(ThriftMux.client().withClientId(clientId));
- } else {
- builder = builder.codec(ThriftClientFramedCodec.apply(Option.apply(clientId)));
- }
-
- Name name;
- try {
- name = Resolver$.MODULE$.eval(serverRoutingServiceFinagleName);
- } catch (Exception exc) {
- logger.error("Exception in Resolver.eval for name {}", serverRoutingServiceFinagleName, exc);
- throw new RuntimeException(exc);
- }
-
- // builder the client
- Service<ThriftClientRequest, byte[]> client =
- ClientBuilder.safeBuildFactory(
- builder.dest(name).reportTo(statsReceiver.scope("routing"))
- ).toService();
- DistributedLogService.ServiceIface service =
- new DistributedLogService.ServiceToClient(client, new TBinaryProtocol.Factory());
- return new ClusterClient(client, service);
- }
-
- DistributedLogClientImpl buildClient() {
- checkNotNull(name, "No name provided.");
- checkNotNull(clientId, "No client id provided.");
- checkNotNull(routingServiceBuilder, "No routing service builder provided.");
- checkNotNull(statsReceiver, "No stats receiver provided.");
- if (null == streamStatsReceiver) {
- streamStatsReceiver = new NullStatsReceiver();
- }
-
- Optional<ClusterClient> serverRoutingServiceClient = Optional.absent();
- if (null != serverRoutingServiceFinagleName) {
- serverRoutingServiceClient = Optional.of(
- buildServerRoutingServiceClient(serverRoutingServiceFinagleName));
- }
-
- RoutingService routingService = routingServiceBuilder
- .statsReceiver(statsReceiver.scope("routing"))
- .build();
- DistributedLogClientImpl clientImpl =
- new DistributedLogClientImpl(
- name,
- clientId,
- routingService,
- clientBuilder,
- clientConfig,
- serverRoutingServiceClient,
- statsReceiver,
- streamStatsReceiver,
- regionResolver,
- enableRegionStats);
- routingService.startService();
- clientImpl.handshake();
- return clientImpl;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/service/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/service/package-info.java
deleted file mode 100644
index e6e56c0..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Service Client.
- */
-package com.twitter.distributedlog.service;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java
new file mode 100644
index 0000000..57e2b5a
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java
@@ -0,0 +1,187 @@
+/**
+ * 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 org.apache.distributedlog.client;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client Config.
+ */
+public class ClientConfig {
+ int redirectBackoffStartMs = 25;
+ int redirectBackoffMaxMs = 100;
+ int maxRedirects = -1;
+ int requestTimeoutMs = -1;
+ boolean thriftmux = false;
+ boolean streamFailfast = false;
+ String streamNameRegex = ".*";
+ boolean handshakeWithClientInfo = true;
+ long periodicHandshakeIntervalMs = TimeUnit.MINUTES.toMillis(5);
+ long periodicOwnershipSyncIntervalMs = TimeUnit.MINUTES.toMillis(5);
+ boolean periodicDumpOwnershipCacheEnabled = false;
+ long periodicDumpOwnershipCacheIntervalMs = TimeUnit.MINUTES.toMillis(10);
+ boolean enableHandshakeTracing = false;
+ boolean enableChecksum = true;
+
+ public ClientConfig setMaxRedirects(int maxRedirects) {
+ this.maxRedirects = maxRedirects;
+ return this;
+ }
+
+ public int getMaxRedirects() {
+ return this.maxRedirects;
+ }
+
+ public ClientConfig setRequestTimeoutMs(int timeoutInMillis) {
+ this.requestTimeoutMs = timeoutInMillis;
+ return this;
+ }
+
+ public int getRequestTimeoutMs() {
+ return this.requestTimeoutMs;
+ }
+
+ public ClientConfig setRedirectBackoffStartMs(int ms) {
+ this.redirectBackoffStartMs = ms;
+ return this;
+ }
+
+ public int getRedirectBackoffStartMs() {
+ return this.redirectBackoffStartMs;
+ }
+
+ public ClientConfig setRedirectBackoffMaxMs(int ms) {
+ this.redirectBackoffMaxMs = ms;
+ return this;
+ }
+
+ public int getRedirectBackoffMaxMs() {
+ return this.redirectBackoffMaxMs;
+ }
+
+ public ClientConfig setThriftMux(boolean enabled) {
+ this.thriftmux = enabled;
+ return this;
+ }
+
+ public boolean getThriftMux() {
+ return this.thriftmux;
+ }
+
+ public ClientConfig setStreamFailfast(boolean enabled) {
+ this.streamFailfast = enabled;
+ return this;
+ }
+
+ public boolean getStreamFailfast() {
+ return this.streamFailfast;
+ }
+
+ public ClientConfig setStreamNameRegex(String nameRegex) {
+ checkNotNull(nameRegex);
+ this.streamNameRegex = nameRegex;
+ return this;
+ }
+
+ public String getStreamNameRegex() {
+ return this.streamNameRegex;
+ }
+
+ public ClientConfig setHandshakeWithClientInfo(boolean enabled) {
+ this.handshakeWithClientInfo = enabled;
+ return this;
+ }
+
+ public boolean getHandshakeWithClientInfo() {
+ return this.handshakeWithClientInfo;
+ }
+
+ public ClientConfig setPeriodicHandshakeIntervalMs(long intervalMs) {
+ this.periodicHandshakeIntervalMs = intervalMs;
+ return this;
+ }
+
+ public long getPeriodicHandshakeIntervalMs() {
+ return this.periodicHandshakeIntervalMs;
+ }
+
+ public ClientConfig setPeriodicOwnershipSyncIntervalMs(long intervalMs) {
+ this.periodicOwnershipSyncIntervalMs = intervalMs;
+ return this;
+ }
+
+ public long getPeriodicOwnershipSyncIntervalMs() {
+ return this.periodicOwnershipSyncIntervalMs;
+ }
+
+ public ClientConfig setPeriodicDumpOwnershipCacheEnabled(boolean enabled) {
+ this.periodicDumpOwnershipCacheEnabled = enabled;
+ return this;
+ }
+
+ public boolean isPeriodicDumpOwnershipCacheEnabled() {
+ return this.periodicDumpOwnershipCacheEnabled;
+ }
+
+ public ClientConfig setPeriodicDumpOwnershipCacheIntervalMs(long intervalMs) {
+ this.periodicDumpOwnershipCacheIntervalMs = intervalMs;
+ return this;
+ }
+
+ public long getPeriodicDumpOwnershipCacheIntervalMs() {
+ return this.periodicDumpOwnershipCacheIntervalMs;
+ }
+
+ public ClientConfig setHandshakeTracingEnabled(boolean enabled) {
+ this.enableHandshakeTracing = enabled;
+ return this;
+ }
+
+ public boolean isHandshakeTracingEnabled() {
+ return this.enableHandshakeTracing;
+ }
+
+ public ClientConfig setChecksumEnabled(boolean enabled) {
+ this.enableChecksum = enabled;
+ return this;
+ }
+
+ public boolean isChecksumEnabled() {
+ return this.enableChecksum;
+ }
+
+ public static ClientConfig newConfig(ClientConfig config) {
+ ClientConfig newConfig = new ClientConfig();
+ newConfig.setMaxRedirects(config.getMaxRedirects())
+ .setRequestTimeoutMs(config.getRequestTimeoutMs())
+ .setRedirectBackoffStartMs(config.getRedirectBackoffStartMs())
+ .setRedirectBackoffMaxMs(config.getRedirectBackoffMaxMs())
+ .setThriftMux(config.getThriftMux())
+ .setStreamFailfast(config.getStreamFailfast())
+ .setStreamNameRegex(config.getStreamNameRegex())
+ .setHandshakeWithClientInfo(config.getHandshakeWithClientInfo())
+ .setPeriodicHandshakeIntervalMs(config.getPeriodicHandshakeIntervalMs())
+ .setPeriodicDumpOwnershipCacheEnabled(config.isPeriodicDumpOwnershipCacheEnabled())
+ .setPeriodicDumpOwnershipCacheIntervalMs(config.getPeriodicDumpOwnershipCacheIntervalMs())
+ .setHandshakeTracingEnabled(config.isHandshakeTracingEnabled())
+ .setChecksumEnabled(config.isChecksumEnabled());
+ return newConfig;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java
new file mode 100644
index 0000000..1300187
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java
@@ -0,0 +1,1200 @@
+/**
+ * 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 org.apache.distributedlog.client;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecordSetBuffer;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.client.ownership.OwnershipCache;
+import org.apache.distributedlog.client.proxy.ClusterClient;
+import org.apache.distributedlog.client.proxy.HostProvider;
+import org.apache.distributedlog.client.proxy.ProxyClient;
+import org.apache.distributedlog.client.proxy.ProxyClientManager;
+import org.apache.distributedlog.client.proxy.ProxyListener;
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.client.routing.RoutingService.RoutingContext;
+import org.apache.distributedlog.client.stats.ClientStats;
+import org.apache.distributedlog.client.stats.OpStats;
+import org.apache.distributedlog.exceptions.DLClientClosedException;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.ServiceUnavailableException;
+import org.apache.distributedlog.exceptions.StreamUnavailableException;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.HeartbeatOptions;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import org.apache.distributedlog.thrift.service.ServerStatus;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import org.apache.distributedlog.thrift.service.WriteContext;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.ProtocolUtils;
+import com.twitter.finagle.CancelledRequestException;
+import com.twitter.finagle.ConnectionFailedException;
+import com.twitter.finagle.Failure;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.RequestTimeoutException;
+import com.twitter.finagle.ServiceException;
+import com.twitter.finagle.ServiceTimeoutException;
+import com.twitter.finagle.WriteException;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId;
+import com.twitter.util.Duration;
+import com.twitter.util.Function;
+import com.twitter.util.Function0;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import com.twitter.util.Return;
+import com.twitter.util.Throw;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.thrift.TApplicationException;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.Seq;
+import scala.runtime.AbstractFunction1;
+
+
+/**
+ * Implementation of distributedlog client.
+ */
+public class DistributedLogClientImpl implements DistributedLogClient, MonitorServiceClient,
+ RoutingService.RoutingListener, ProxyListener, HostProvider {
+
+ private static final Logger logger = LoggerFactory.getLogger(DistributedLogClientImpl.class);
+
+ private final String clientName;
+ private final ClientId clientId;
+ private final ClientConfig clientConfig;
+ private final RoutingService routingService;
+ private final ProxyClient.Builder clientBuilder;
+ private final boolean streamFailfast;
+ private final Pattern streamNameRegexPattern;
+
+ // Timer
+ private final HashedWheelTimer dlTimer;
+
+ // region resolver
+ private final RegionResolver regionResolver;
+
+ // Ownership maintenance
+ private final OwnershipCache ownershipCache;
+ // Channel/Client management
+ private final ProxyClientManager clientManager;
+ // Cluster Client (for routing service)
+ private final Optional<ClusterClient> clusterClient;
+
+ // Close Status
+ private boolean closed = false;
+ private final ReentrantReadWriteLock closeLock =
+ new ReentrantReadWriteLock();
+
+ abstract class StreamOp implements TimerTask {
+ final String stream;
+
+ final AtomicInteger tries = new AtomicInteger(0);
+ final RoutingContext routingContext = RoutingContext.of(regionResolver);
+ final WriteContext ctx = new WriteContext();
+ final Stopwatch stopwatch;
+ final OpStats opStats;
+ SocketAddress nextAddressToSend;
+
+ StreamOp(final String stream, final OpStats opStats) {
+ this.stream = stream;
+ this.stopwatch = Stopwatch.createStarted();
+ this.opStats = opStats;
+ }
+
+ boolean shouldTimeout() {
+ long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
+ return shouldTimeout(elapsedMs);
+ }
+
+ boolean shouldTimeout(long elapsedMs) {
+ return clientConfig.getRequestTimeoutMs() > 0
+ && elapsedMs >= clientConfig.getRequestTimeoutMs();
+ }
+
+ void send(SocketAddress address) {
+ long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
+ if (clientConfig.getMaxRedirects() > 0
+ && tries.get() >= clientConfig.getMaxRedirects()) {
+ fail(address, new RequestTimeoutException(Duration.fromMilliseconds(elapsedMs),
+ "Exhausted max redirects in " + elapsedMs + " ms"));
+ return;
+ } else if (shouldTimeout(elapsedMs)) {
+ fail(address, new RequestTimeoutException(Duration.fromMilliseconds(elapsedMs),
+ "Exhausted max request timeout " + clientConfig.getRequestTimeoutMs()
+ + " in " + elapsedMs + " ms"));
+ return;
+ }
+ synchronized (this) {
+ String addrStr = address.toString();
+ if (ctx.isSetTriedHosts() && ctx.getTriedHosts().contains(addrStr)) {
+ nextAddressToSend = address;
+ dlTimer.newTimeout(this,
+ Math.min(clientConfig.getRedirectBackoffMaxMs(),
+ tries.get() * clientConfig.getRedirectBackoffStartMs()),
+ TimeUnit.MILLISECONDS);
+ } else {
+ doSend(address);
+ }
+ }
+ }
+
+ abstract Future<ResponseHeader> sendRequest(ProxyClient sc);
+
+ void doSend(SocketAddress address) {
+ ctx.addToTriedHosts(address.toString());
+ if (clientConfig.isChecksumEnabled()) {
+ Long crc32 = computeChecksum();
+ if (null != crc32) {
+ ctx.setCrc32(crc32);
+ }
+ }
+ tries.incrementAndGet();
+ sendWriteRequest(address, this);
+ }
+
+ void beforeComplete(ProxyClient sc, ResponseHeader responseHeader) {
+ ownershipCache.updateOwner(stream, sc.getAddress());
+ }
+
+ void complete(SocketAddress address) {
+ stopwatch.stop();
+ opStats.completeRequest(address,
+ stopwatch.elapsed(TimeUnit.MICROSECONDS), tries.get());
+ }
+
+ void fail(SocketAddress address, Throwable t) {
+ stopwatch.stop();
+ opStats.failRequest(address,
+ stopwatch.elapsed(TimeUnit.MICROSECONDS), tries.get());
+ }
+
+ Long computeChecksum() {
+ return null;
+ }
+
+ @Override
+ public synchronized void run(Timeout timeout) throws Exception {
+ if (!timeout.isCancelled() && null != nextAddressToSend) {
+ doSend(nextAddressToSend);
+ } else {
+ fail(null, new CancelledRequestException());
+ }
+ }
+ }
+
+ class BulkWriteOp extends StreamOp {
+
+ final List<ByteBuffer> data;
+ final ArrayList<Promise<DLSN>> results;
+
+ BulkWriteOp(final String name, final List<ByteBuffer> data) {
+ super(name, clientStats.getOpStats("bulk_write"));
+ this.data = data;
+
+ // This could take a while (relatively speaking) for very large inputs. We probably don't want
+ // to go so large for other reasons though.
+ this.results = new ArrayList<Promise<DLSN>>(data.size());
+ for (int i = 0; i < data.size(); i++) {
+ checkNotNull(data.get(i));
+ this.results.add(new Promise<DLSN>());
+ }
+ }
+
+ @Override
+ Future<ResponseHeader> sendRequest(final ProxyClient sc) {
+ return sc.getService().writeBulkWithContext(stream, data, ctx)
+ .addEventListener(new FutureEventListener<BulkWriteResponse>() {
+ @Override
+ public void onSuccess(BulkWriteResponse response) {
+ // For non-success case, the ResponseHeader handler (the caller) will handle it.
+ // Note success in this case means no finagle errors have occurred
+ // (such as finagle connection issues). In general code != SUCCESS means there's some error
+ // reported by dlog service. The caller will handle such errors.
+ if (response.getHeader().getCode() == StatusCode.SUCCESS) {
+ beforeComplete(sc, response.getHeader());
+ BulkWriteOp.this.complete(sc.getAddress(), response);
+ if (response.getWriteResponses().size() == 0 && data.size() > 0) {
+ logger.error("non-empty bulk write got back empty response without failure for stream {}",
+ stream);
+ }
+ }
+ }
+ @Override
+ public void onFailure(Throwable cause) {
+ // Handled by the ResponseHeader listener (attached by the caller).
+ }
+ }).map(new AbstractFunction1<BulkWriteResponse, ResponseHeader>() {
+ @Override
+ public ResponseHeader apply(BulkWriteResponse response) {
+ // We need to return the ResponseHeader to the caller's listener to process DLOG errors.
+ return response.getHeader();
+ }
+ });
+ }
+
+ void complete(SocketAddress address, BulkWriteResponse bulkWriteResponse) {
+ super.complete(address);
+ Iterator<WriteResponse> writeResponseIterator = bulkWriteResponse.getWriteResponses().iterator();
+ Iterator<Promise<DLSN>> resultIterator = results.iterator();
+
+ // Fill in errors from thrift responses.
+ while (resultIterator.hasNext() && writeResponseIterator.hasNext()) {
+ Promise<DLSN> result = resultIterator.next();
+ WriteResponse writeResponse = writeResponseIterator.next();
+ if (StatusCode.SUCCESS == writeResponse.getHeader().getCode()) {
+ result.setValue(DLSN.deserialize(writeResponse.getDlsn()));
+ } else {
+ result.setException(DLException.of(writeResponse.getHeader()));
+ }
+ }
+
+ // Should never happen, but just in case so there's some record.
+ if (bulkWriteResponse.getWriteResponses().size() != data.size()) {
+ logger.error("wrong number of results, response = {} records = {}",
+ bulkWriteResponse.getWriteResponses().size(), data.size());
+ }
+ }
+
+ @Override
+ void fail(SocketAddress address, Throwable t) {
+
+ // StreamOp.fail is called to fail the overall request. In case of BulkWriteOp we take the request level
+ // exception to apply to the first write. In fact for request level exceptions no request has ever been
+ // attempted, but logically we associate the error with the first write.
+ super.fail(address, t);
+ Iterator<Promise<DLSN>> resultIterator = results.iterator();
+
+ // Fail the first write with the batch level failure.
+ if (resultIterator.hasNext()) {
+ Promise<DLSN> result = resultIterator.next();
+ result.setException(t);
+ }
+
+ // Fail the remaining writes as cancelled requests.
+ while (resultIterator.hasNext()) {
+ Promise<DLSN> result = resultIterator.next();
+ result.setException(new CancelledRequestException());
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ List<Future<DLSN>> result() {
+ return (List) results;
+ }
+ }
+
+ abstract class AbstractWriteOp extends StreamOp {
+
+ final Promise<WriteResponse> result = new Promise<WriteResponse>();
+ Long crc32 = null;
+
+ AbstractWriteOp(final String name, final OpStats opStats) {
+ super(name, opStats);
+ }
+
+ void complete(SocketAddress address, WriteResponse response) {
+ super.complete(address);
+ result.setValue(response);
+ }
+
+ @Override
+ void fail(SocketAddress address, Throwable t) {
+ super.fail(address, t);
+ result.setException(t);
+ }
+
+ @Override
+ Long computeChecksum() {
+ if (null == crc32) {
+ crc32 = ProtocolUtils.streamOpCRC32(stream);
+ }
+ return crc32;
+ }
+
+ @Override
+ Future<ResponseHeader> sendRequest(final ProxyClient sc) {
+ return this.sendWriteRequest(sc).addEventListener(new FutureEventListener<WriteResponse>() {
+ @Override
+ public void onSuccess(WriteResponse response) {
+ if (response.getHeader().getCode() == StatusCode.SUCCESS) {
+ beforeComplete(sc, response.getHeader());
+ AbstractWriteOp.this.complete(sc.getAddress(), response);
+ }
+ }
+ @Override
+ public void onFailure(Throwable cause) {
+ // handled by the ResponseHeader listener
+ }
+ }).map(new AbstractFunction1<WriteResponse, ResponseHeader>() {
+ @Override
+ public ResponseHeader apply(WriteResponse response) {
+ return response.getHeader();
+ }
+ });
+ }
+
+ abstract Future<WriteResponse> sendWriteRequest(ProxyClient sc);
+ }
+
+ class WriteOp extends AbstractWriteOp {
+ final ByteBuffer data;
+
+ WriteOp(final String name, final ByteBuffer data) {
+ super(name, clientStats.getOpStats("write"));
+ this.data = data;
+ }
+
+ @Override
+ Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+ return sc.getService().writeWithContext(stream, data, ctx);
+ }
+
+ @Override
+ Long computeChecksum() {
+ if (null == crc32) {
+ byte[] dataBytes = new byte[data.remaining()];
+ data.duplicate().get(dataBytes);
+ crc32 = ProtocolUtils.writeOpCRC32(stream, dataBytes);
+ }
+ return crc32;
+ }
+
+ Future<DLSN> result() {
+ return result.map(new AbstractFunction1<WriteResponse, DLSN>() {
+ @Override
+ public DLSN apply(WriteResponse response) {
+ return DLSN.deserialize(response.getDlsn());
+ }
+ });
+ }
+ }
+
+ class TruncateOp extends AbstractWriteOp {
+ final DLSN dlsn;
+
+ TruncateOp(String name, DLSN dlsn) {
+ super(name, clientStats.getOpStats("truncate"));
+ this.dlsn = dlsn;
+ }
+
+ @Override
+ Long computeChecksum() {
+ if (null == crc32) {
+ crc32 = ProtocolUtils.truncateOpCRC32(stream, dlsn);
+ }
+ return crc32;
+ }
+
+ @Override
+ Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+ return sc.getService().truncate(stream, dlsn.serialize(), ctx);
+ }
+
+ Future<Boolean> result() {
+ return result.map(new AbstractFunction1<WriteResponse, Boolean>() {
+ @Override
+ public Boolean apply(WriteResponse response) {
+ return true;
+ }
+ });
+ }
+ }
+
+ class WriteRecordSetOp extends WriteOp {
+
+ WriteRecordSetOp(String name, LogRecordSetBuffer recordSet) {
+ super(name, recordSet.getBuffer());
+ ctx.setIsRecordSet(true);
+ }
+
+ }
+
+
+ class ReleaseOp extends AbstractWriteOp {
+
+ ReleaseOp(String name) {
+ super(name, clientStats.getOpStats("release"));
+ }
+
+ @Override
+ Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+ return sc.getService().release(stream, ctx);
+ }
+
+ @Override
+ void beforeComplete(ProxyClient sc, ResponseHeader header) {
+ ownershipCache.removeOwnerFromStream(stream, sc.getAddress(), "Stream Deleted");
+ }
+
+ Future<Void> result() {
+ return result.map(new AbstractFunction1<WriteResponse, Void>() {
+ @Override
+ public Void apply(WriteResponse response) {
+ return null;
+ }
+ });
+ }
+ }
+
+ class DeleteOp extends AbstractWriteOp {
+
+ DeleteOp(String name) {
+ super(name, clientStats.getOpStats("delete"));
+ }
+
+ @Override
+ Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+ return sc.getService().delete(stream, ctx);
+ }
+
+ @Override
+ void beforeComplete(ProxyClient sc, ResponseHeader header) {
+ ownershipCache.removeOwnerFromStream(stream, sc.getAddress(), "Stream Deleted");
+ }
+
+ Future<Void> result() {
+ return result.map(new AbstractFunction1<WriteResponse, Void>() {
+ @Override
+ public Void apply(WriteResponse v1) {
+ return null;
+ }
+ });
+ }
+ }
+
+ class CreateOp extends AbstractWriteOp {
+
+ CreateOp(String name) {
+ super(name, clientStats.getOpStats("create"));
+ }
+
+ @Override
+ Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+ return sc.getService().create(stream, ctx);
+ }
+
+ @Override
+ void beforeComplete(ProxyClient sc, ResponseHeader header) {
+ ownershipCache.updateOwner(stream, sc.getAddress());
+ }
+
+ Future<Void> result() {
+ return result.map(new AbstractFunction1<WriteResponse, Void>() {
+ @Override
+ public Void apply(WriteResponse v1) {
+ return null;
+ }
+ }).voided();
+ }
+ }
+
+ class HeartbeatOp extends AbstractWriteOp {
+ HeartbeatOptions options;
+
+ HeartbeatOp(String name, boolean sendReaderHeartBeat) {
+ super(name, clientStats.getOpStats("heartbeat"));
+ options = new HeartbeatOptions();
+ options.setSendHeartBeatToReader(sendReaderHeartBeat);
+ }
+
+ @Override
+ Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+ return sc.getService().heartbeatWithOptions(stream, ctx, options);
+ }
+
+ Future<Void> result() {
+ return result.map(new AbstractFunction1<WriteResponse, Void>() {
+ @Override
+ public Void apply(WriteResponse response) {
+ return null;
+ }
+ });
+ }
+ }
+
+ // Stats
+ private final ClientStats clientStats;
+
+ public DistributedLogClientImpl(String name,
+ ClientId clientId,
+ RoutingService routingService,
+ ClientBuilder clientBuilder,
+ ClientConfig clientConfig,
+ Optional<ClusterClient> clusterClient,
+ StatsReceiver statsReceiver,
+ StatsReceiver streamStatsReceiver,
+ RegionResolver regionResolver,
+ boolean enableRegionStats) {
+ this.clientName = name;
+ this.clientId = clientId;
+ this.routingService = routingService;
+ this.clientConfig = clientConfig;
+ this.streamFailfast = clientConfig.getStreamFailfast();
+ this.streamNameRegexPattern = Pattern.compile(clientConfig.getStreamNameRegex());
+ this.regionResolver = regionResolver;
+ // Build the timer
+ this.dlTimer = new HashedWheelTimer(
+ new ThreadFactoryBuilder().setNameFormat("DLClient-" + name + "-timer-%d").build(),
+ this.clientConfig.getRedirectBackoffStartMs(),
+ TimeUnit.MILLISECONDS);
+ // register routing listener
+ this.routingService.registerListener(this);
+ // build the ownership cache
+ this.ownershipCache = new OwnershipCache(this.clientConfig, this.dlTimer, statsReceiver, streamStatsReceiver);
+ // Client Stats
+ this.clientStats = new ClientStats(statsReceiver, enableRegionStats, regionResolver);
+ // Client Manager
+ this.clientBuilder = ProxyClient.newBuilder(clientName, clientId, clientBuilder, clientConfig, clientStats);
+ this.clientManager = new ProxyClientManager(
+ this.clientConfig, // client config
+ this.clientBuilder, // client builder
+ this.dlTimer, // timer
+ this, // host provider
+ clientStats); // client stats
+ this.clusterClient = clusterClient;
+ this.clientManager.registerProxyListener(this);
+
+ // Cache Stats
+ StatsReceiver cacheStatReceiver = statsReceiver.scope("cache");
+ Seq<String> numCachedStreamsGaugeName =
+ scala.collection.JavaConversions.asScalaBuffer(Arrays.asList("num_streams")).toList();
+ cacheStatReceiver.provideGauge(numCachedStreamsGaugeName, new Function0<Object>() {
+ @Override
+ public Object apply() {
+ return (float) ownershipCache.getNumCachedStreams();
+ }
+ });
+ Seq<String> numCachedHostsGaugeName =
+ scala.collection.JavaConversions.asScalaBuffer(Arrays.asList("num_hosts")).toList();
+ cacheStatReceiver.provideGauge(numCachedHostsGaugeName, new Function0<Object>() {
+ @Override
+ public Object apply() {
+ return (float) clientManager.getNumProxies();
+ }
+ });
+
+ logger.info("Build distributedlog client : name = {}, client_id = {}, routing_service = {},"
+ + " stats_receiver = {}, thriftmux = {}",
+ new Object[] {
+ name,
+ clientId,
+ routingService.getClass(),
+ statsReceiver.getClass(),
+ clientConfig.getThriftMux()
+ });
+ }
+
+ @Override
+ public Set<SocketAddress> getHosts() {
+ Set<SocketAddress> hosts = Sets.newHashSet();
+ // if using server side routing, we only handshake with the hosts in ownership cache.
+ if (!clusterClient.isPresent()) {
+ hosts.addAll(this.routingService.getHosts());
+ }
+ hosts.addAll(this.ownershipCache.getStreamOwnershipDistribution().keySet());
+ return hosts;
+ }
+
+ @Override
+ public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
+ if (null != serverInfo
+ && serverInfo.isSetServerStatus()
+ && ServerStatus.DOWN == serverInfo.getServerStatus()) {
+ logger.info("{} is detected as DOWN during handshaking", address);
+ // server is shutting down
+ handleServiceUnavailable(address, client, Optional.<StreamOp>absent());
+ return;
+ }
+
+ if (null != serverInfo && serverInfo.isSetOwnerships()) {
+ Map<String, String> ownerships = serverInfo.getOwnerships();
+ logger.debug("Handshaked with {} : {} ownerships returned.", address, ownerships.size());
+ for (Map.Entry<String, String> entry : ownerships.entrySet()) {
+ Matcher matcher = streamNameRegexPattern.matcher(entry.getKey());
+ if (!matcher.matches()) {
+ continue;
+ }
+ updateOwnership(entry.getKey(), entry.getValue());
+ }
+ } else {
+ logger.debug("Handshaked with {} : no ownerships returned", address);
+ }
+ }
+
+ @Override
+ public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
+ cause = showRootCause(Optional.<StreamOp>absent(), cause);
+ handleRequestException(address, client, Optional.<StreamOp>absent(), cause);
+ }
+
+ @VisibleForTesting
+ public void handshake() {
+ clientManager.handshake();
+ logger.info("Handshaked with {} hosts, cached {} streams",
+ clientManager.getNumProxies(), ownershipCache.getNumCachedStreams());
+ }
+
+ @Override
+ public void onServerLeft(SocketAddress address) {
+ onServerLeft(address, null);
+ }
+
+ private void onServerLeft(SocketAddress address, ProxyClient sc) {
+ ownershipCache.removeAllStreamsFromOwner(address);
+ if (null == sc) {
+ clientManager.removeClient(address);
+ } else {
+ clientManager.removeClient(address, sc);
+ }
+ }
+
+ @Override
+ public void onServerJoin(SocketAddress address) {
+ // we only pre-create connection for client-side routing
+ // if it is server side routing, we only know the exact proxy address
+ // when #getOwner.
+ if (!clusterClient.isPresent()) {
+ clientManager.createClient(address);
+ }
+ }
+
+ public void close() {
+ closeLock.writeLock().lock();
+ try {
+ if (closed) {
+ return;
+ }
+ closed = true;
+ } finally {
+ closeLock.writeLock().unlock();
+ }
+ clientManager.close();
+ routingService.unregisterListener(this);
+ routingService.stopService();
+ dlTimer.stop();
+ }
+
+ @Override
+ public Future<Void> check(String stream) {
+ final HeartbeatOp op = new HeartbeatOp(stream, false);
+ sendRequest(op);
+ return op.result();
+ }
+
+ @Override
+ public Future<Void> heartbeat(String stream) {
+ final HeartbeatOp op = new HeartbeatOp(stream, true);
+ sendRequest(op);
+ return op.result();
+ }
+
+ @Override
+ public Map<SocketAddress, Set<String>> getStreamOwnershipDistribution() {
+ return ownershipCache.getStreamOwnershipDistribution();
+ }
+
+ @Override
+ public Future<Void> setAcceptNewStream(boolean enabled) {
+ Map<SocketAddress, ProxyClient> snapshot = clientManager.getAllClients();
+ List<Future<Void>> futures = new ArrayList<Future<Void>>(snapshot.size());
+ for (Map.Entry<SocketAddress, ProxyClient> entry : snapshot.entrySet()) {
+ futures.add(entry.getValue().getService().setAcceptNewStream(enabled));
+ }
+ return Future.collect(futures).map(new Function<List<Void>, Void>() {
+ @Override
+ public Void apply(List<Void> list) {
+ return null;
+ }
+ });
+ }
+
+ @Override
+ public Future<DLSN> write(String stream, ByteBuffer data) {
+ final WriteOp op = new WriteOp(stream, data);
+ sendRequest(op);
+ return op.result();
+ }
+
+ @Override
+ public Future<DLSN> writeRecordSet(String stream, final LogRecordSetBuffer recordSet) {
+ final WriteRecordSetOp op = new WriteRecordSetOp(stream, recordSet);
+ sendRequest(op);
+ return op.result();
+ }
+
+ @Override
+ public List<Future<DLSN>> writeBulk(String stream, List<ByteBuffer> data) {
+ if (data.size() > 0) {
+ final BulkWriteOp op = new BulkWriteOp(stream, data);
+ sendRequest(op);
+ return op.result();
+ } else {
+ return Collections.emptyList();
+ }
+ }
+
+ @Override
+ public Future<Boolean> truncate(String stream, DLSN dlsn) {
+ final TruncateOp op = new TruncateOp(stream, dlsn);
+ sendRequest(op);
+ return op.result();
+ }
+
+ @Override
+ public Future<Void> delete(String stream) {
+ final DeleteOp op = new DeleteOp(stream);
+ sendRequest(op);
+ return op.result();
+ }
+
+ @Override
+ public Future<Void> release(String stream) {
+ final ReleaseOp op = new ReleaseOp(stream);
+ sendRequest(op);
+ return op.result();
+ }
+
+ @Override
+ public Future<Void> create(String stream) {
+ final CreateOp op = new CreateOp(stream);
+ sendRequest(op);
+ return op.result();
+ }
+
+ private void sendRequest(final StreamOp op) {
+ closeLock.readLock().lock();
+ try {
+ if (closed) {
+ op.fail(null, new DLClientClosedException("Client " + clientName + " is closed."));
+ } else {
+ doSend(op, null);
+ }
+ } finally {
+ closeLock.readLock().unlock();
+ }
+ }
+
+ /**
+ * Send the stream operation by routing service, excluding previous address if it is not null.
+ *
+ * @param op
+ * stream operation.
+ * @param previousAddr
+ * previous tried address.
+ */
+ private void doSend(final StreamOp op, final SocketAddress previousAddr) {
+ if (null != previousAddr) {
+ op.routingContext.addTriedHost(previousAddr, StatusCode.WRITE_EXCEPTION);
+ }
+ // Get host first
+ final SocketAddress address = ownershipCache.getOwner(op.stream);
+ if (null == address || op.routingContext.isTriedHost(address)) {
+ getOwner(op).addEventListener(new FutureEventListener<SocketAddress>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ op.fail(null, cause);
+ }
+
+ @Override
+ public void onSuccess(SocketAddress ownerAddr) {
+ op.send(ownerAddr);
+ }
+ });
+ } else {
+ op.send(address);
+ }
+ }
+
+ private void retryGetOwnerFromResourcePlacementServer(final StreamOp op,
+ final Promise<SocketAddress> getOwnerPromise,
+ final Throwable cause) {
+ if (op.shouldTimeout()) {
+ op.fail(null, cause);
+ return;
+ }
+ getOwnerFromResourcePlacementServer(op, getOwnerPromise);
+ }
+
+ private void getOwnerFromResourcePlacementServer(final StreamOp op,
+ final Promise<SocketAddress> getOwnerPromise) {
+ clusterClient.get().getService().getOwner(op.stream, op.ctx)
+ .addEventListener(new FutureEventListener<WriteResponse>() {
+ @Override
+ public void onFailure(Throwable cause) {
+ getOwnerPromise.updateIfEmpty(new Throw<SocketAddress>(cause));
+ }
+
+ @Override
+ public void onSuccess(WriteResponse value) {
+ if (StatusCode.FOUND == value.getHeader().getCode()
+ && null != value.getHeader().getLocation()) {
+ try {
+ InetSocketAddress addr = DLSocketAddress.deserialize(
+ value.getHeader().getLocation()
+ ).getSocketAddress();
+ getOwnerPromise.updateIfEmpty(new Return<SocketAddress>(addr));
+ } catch (IOException e) {
+ // retry from the routing server again
+ logger.error("ERROR in getOwner", e);
+ retryGetOwnerFromResourcePlacementServer(op, getOwnerPromise, e);
+ return;
+ }
+ } else {
+ // retry from the routing server again
+ retryGetOwnerFromResourcePlacementServer(op, getOwnerPromise,
+ new StreamUnavailableException("Stream " + op.stream + "'s owner is unknown"));
+ }
+ }
+ });
+ }
+
+ private Future<SocketAddress> getOwner(final StreamOp op) {
+ if (clusterClient.isPresent()) {
+ final Promise<SocketAddress> getOwnerPromise = new Promise<SocketAddress>();
+ getOwnerFromResourcePlacementServer(op, getOwnerPromise);
+ return getOwnerPromise;
+ }
+ // pickup host by hashing
+ try {
+ return Future.value(routingService.getHost(op.stream, op.routingContext));
+ } catch (NoBrokersAvailableException nbae) {
+ return Future.exception(nbae);
+ }
+ }
+
+ private void sendWriteRequest(final SocketAddress addr, final StreamOp op) {
+ // Get corresponding finagle client
+ final ProxyClient sc = clientManager.getClient(addr);
+ final long startTimeNanos = System.nanoTime();
+ // write the request to that host.
+ op.sendRequest(sc).addEventListener(new FutureEventListener<ResponseHeader>() {
+ @Override
+ public void onSuccess(ResponseHeader header) {
+ if (logger.isDebugEnabled()) {
+ logger.debug("Received response; header: {}", header);
+ }
+ clientStats.completeProxyRequest(addr, header.getCode(), startTimeNanos);
+ // update routing context
+ op.routingContext.addTriedHost(addr, header.getCode());
+ switch (header.getCode()) {
+ case SUCCESS:
+ // success handling is done per stream op
+ break;
+ case FOUND:
+ handleRedirectResponse(header, op, addr);
+ break;
+ // for overcapacity, dont report failure since this normally happens quite a bit
+ case OVER_CAPACITY:
+ logger.debug("Failed to write request to {} : {}", op.stream, header);
+ op.fail(addr, DLException.of(header));
+ break;
+ // for responses that indicate the requests definitely failed,
+ // we should fail them immediately (e.g. TOO_LARGE_RECORD, METADATA_EXCEPTION)
+ case NOT_IMPLEMENTED:
+ case METADATA_EXCEPTION:
+ case LOG_EMPTY:
+ case LOG_NOT_FOUND:
+ case TRUNCATED_TRANSACTION:
+ case END_OF_STREAM:
+ case TRANSACTION_OUT_OF_ORDER:
+ case INVALID_STREAM_NAME:
+ case REQUEST_DENIED:
+ case TOO_LARGE_RECORD:
+ case CHECKSUM_FAILED:
+ // status code NOT_READY is returned if failfast is enabled in the server. don't redirect
+ // since the proxy may still own the stream.
+ case STREAM_NOT_READY:
+ op.fail(addr, DLException.of(header));
+ break;
+ case SERVICE_UNAVAILABLE:
+ handleServiceUnavailable(addr, sc, Optional.of(op));
+ break;
+ case REGION_UNAVAILABLE:
+ // region is unavailable, redirect the request to hosts in other region
+ redirect(op, null);
+ break;
+ // Proxy was overloaded and refused to try to acquire the stream. Don't remove ownership, since
+ // we didn't have it in the first place.
+ case TOO_MANY_STREAMS:
+ handleRedirectableError(addr, op, header);
+ break;
+ case STREAM_UNAVAILABLE:
+ case ZOOKEEPER_ERROR:
+ case LOCKING_EXCEPTION:
+ case UNEXPECTED:
+ case INTERRUPTED:
+ case BK_TRANSMIT_ERROR:
+ case FLUSH_TIMEOUT:
+ default:
+ // when we are receiving these exceptions from proxy, it means proxy or the stream is closed
+ // redirect the request.
+ ownershipCache.removeOwnerFromStream(op.stream, addr, header.getCode().name());
+ handleRedirectableError(addr, op, header);
+ break;
+ }
+ }
+
+ @Override
+ public void onFailure(Throwable cause) {
+ Optional<StreamOp> opOptional = Optional.of(op);
+ cause = showRootCause(opOptional, cause);
+ clientStats.failProxyRequest(addr, cause, startTimeNanos);
+ handleRequestException(addr, sc, opOptional, cause);
+ }
+ });
+ }
+
+ // Response Handlers
+
+ Throwable showRootCause(Optional<StreamOp> op, Throwable cause) {
+ if (cause instanceof Failure) {
+ Failure failure = (Failure) cause;
+ if (failure.isFlagged(Failure.Wrapped())) {
+ try {
+ // if it is a wrapped failure, unwrap it first
+ cause = failure.show();
+ } catch (IllegalArgumentException iae) {
+ if (op.isPresent()) {
+ logger.warn("Failed to unwrap finagle failure of stream {} : ", op.get().stream, iae);
+ } else {
+ logger.warn("Failed to unwrap finagle failure : ", iae);
+ }
+ }
+ }
+ }
+ return cause;
+ }
+
+ private void handleRedirectableError(SocketAddress addr,
+ StreamOp op,
+ ResponseHeader header) {
+ if (streamFailfast) {
+ op.fail(addr, DLException.of(header));
+ } else {
+ redirect(op, null);
+ }
+ }
+
+ void handleServiceUnavailable(SocketAddress addr,
+ ProxyClient sc,
+ Optional<StreamOp> op) {
+ // service is unavailable, remove it out of routing service
+ routingService.removeHost(addr, new ServiceUnavailableException(addr + " is unavailable now."));
+ onServerLeft(addr);
+ if (op.isPresent()) {
+ ownershipCache.removeOwnerFromStream(op.get().stream, addr, addr + " is unavailable now.");
+ // redirect the request to other host.
+ redirect(op.get(), null);
+ }
+ }
+
+ void handleRequestException(SocketAddress addr,
+ ProxyClient sc,
+ Optional<StreamOp> op,
+ Throwable cause) {
+ boolean resendOp = false;
+ boolean removeOwnerFromStream = false;
+ SocketAddress previousAddr = addr;
+ String reason = cause.getMessage();
+ if (cause instanceof ConnectionFailedException || cause instanceof java.net.ConnectException) {
+ routingService.removeHost(addr, cause);
+ onServerLeft(addr, sc);
+ removeOwnerFromStream = true;
+ // redirect the request to other host.
+ resendOp = true;
+ } else if (cause instanceof ChannelException) {
+ // java.net.ConnectException typically means connection is refused remotely
+ // no process listening on remote address/port.
+ if (cause.getCause() instanceof java.net.ConnectException) {
+ routingService.removeHost(addr, cause.getCause());
+ onServerLeft(addr);
+ reason = cause.getCause().getMessage();
+ } else {
+ routingService.removeHost(addr, cause);
+ reason = cause.getMessage();
+ }
+ removeOwnerFromStream = true;
+ // redirect the request to other host.
+ resendOp = true;
+ } else if (cause instanceof ServiceTimeoutException) {
+ // redirect the request to itself again, which will backoff for a while
+ resendOp = true;
+ previousAddr = null;
+ } else if (cause instanceof WriteException) {
+ // redirect the request to other host.
+ resendOp = true;
+ } else if (cause instanceof ServiceException) {
+ // redirect the request to other host.
+ clientManager.removeClient(addr, sc);
+ resendOp = true;
+ } else if (cause instanceof TApplicationException) {
+ handleTApplicationException(cause, op, addr, sc);
+ } else if (cause instanceof Failure) {
+ handleFinagleFailure((Failure) cause, op, addr);
+ } else {
+ // Default handler
+ handleException(cause, op, addr);
+ }
+
+ if (op.isPresent()) {
+ if (removeOwnerFromStream) {
+ ownershipCache.removeOwnerFromStream(op.get().stream, addr, reason);
+ }
+ if (resendOp) {
+ doSend(op.get(), previousAddr);
+ }
+ }
+ }
+
+ /**
+ * Redirect the request to new proxy <i>newAddr</i>. If <i>newAddr</i> is null,
+ * it would pick up a host from routing service.
+ *
+ * @param op
+ * stream operation
+ * @param newAddr
+ * new proxy address
+ */
+ void redirect(StreamOp op, SocketAddress newAddr) {
+ ownershipCache.getOwnershipStatsLogger().onRedirect(op.stream);
+ if (null != newAddr) {
+ logger.debug("Redirect request {} to new owner {}.", op, newAddr);
+ op.send(newAddr);
+ } else {
+ doSend(op, null);
+ }
+ }
+
+ void handleFinagleFailure(Failure failure,
+ Optional<StreamOp> op,
+ SocketAddress addr) {
+ if (failure.isFlagged(Failure.Restartable())) {
+ if (op.isPresent()) {
+ // redirect the request to other host
+ doSend(op.get(), addr);
+ }
+ } else {
+ // fail the request if it is other types of failures
+ handleException(failure, op, addr);
+ }
+ }
+
+ void handleException(Throwable cause,
+ Optional<StreamOp> op,
+ SocketAddress addr) {
+ // RequestTimeoutException: fail it and let client decide whether to retry or not.
+
+ // FailedFastException:
+ // We don't actually know when FailedFastException will be thrown
+ // so properly we just throw it back to application to let application
+ // handle it.
+
+ // Other Exceptions: as we don't know how to handle them properly so throw them to client
+ if (op.isPresent()) {
+ logger.error("Failed to write request to {} @ {} : {}",
+ new Object[]{op.get().stream, addr, cause.toString()});
+ op.get().fail(addr, cause);
+ }
+ }
+
+ void handleTApplicationException(Throwable cause,
+ Optional<StreamOp> op,
+ SocketAddress addr,
+ ProxyClient sc) {
+ TApplicationException ex = (TApplicationException) cause;
+ if (ex.getType() == TApplicationException.UNKNOWN_METHOD) {
+ // if we encountered unknown method exception on thrift server, it means this proxy
+ // has problem. we should remove it from routing service, clean up ownerships
+ routingService.removeHost(addr, cause);
+ onServerLeft(addr, sc);
+ if (op.isPresent()) {
+ ownershipCache.removeOwnerFromStream(op.get().stream, addr, cause.getMessage());
+ doSend(op.get(), addr);
+ }
+ } else {
+ handleException(cause, op, addr);
+ }
+ }
+
+ void handleRedirectResponse(ResponseHeader header, StreamOp op, SocketAddress curAddr) {
+ SocketAddress ownerAddr = null;
+ if (header.isSetLocation()) {
+ String owner = header.getLocation();
+ try {
+ ownerAddr = DLSocketAddress.deserialize(owner).getSocketAddress();
+ // if we are receiving a direct request to same host, we won't try the same host.
+ // as the proxy will shut itself down if it redirects client to itself.
+ if (curAddr.equals(ownerAddr)) {
+ logger.warn("Request to stream {} is redirected to same server {}!", op.stream, curAddr);
+ ownerAddr = null;
+ } else {
+ // update ownership when redirects.
+ ownershipCache.updateOwner(op.stream, ownerAddr);
+ }
+ } catch (IOException e) {
+ ownerAddr = null;
+ }
+ }
+ redirect(op, ownerAddr);
+ }
+
+ void updateOwnership(String stream, String location) {
+ try {
+ SocketAddress ownerAddr = DLSocketAddress.deserialize(location).getSocketAddress();
+ // update ownership
+ ownershipCache.updateOwner(stream, ownerAddr);
+ } catch (IOException e) {
+ logger.warn("Invalid ownership {} found for stream {} : ",
+ new Object[] { location, stream, e });
+ }
+ }
+
+}
[46/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClient.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClient.java
deleted file mode 100644
index d131e28..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClient.java
+++ /dev/null
@@ -1,165 +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.client.proxy;
-
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.stats.ClientStats;
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.finagle.Service;
-import com.twitter.finagle.ThriftMux;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId;
-import com.twitter.finagle.thrift.ThriftClientFramedCodec;
-import com.twitter.finagle.thrift.ThriftClientRequest;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import scala.Option;
-import scala.runtime.BoxedUnit;
-
-/**
- * Client talks to a single proxy.
- */
-public class ProxyClient {
-
- /**
- * Builder to build a proxy client talking to given host <code>address</code>.
- */
- public interface Builder {
- /**
- * Build a proxy client to <code>address</code>.
- *
- * @param address
- * proxy address
- * @return proxy client
- */
- ProxyClient build(SocketAddress address);
- }
-
- public static Builder newBuilder(String clientName,
- ClientId clientId,
- ClientBuilder clientBuilder,
- ClientConfig clientConfig,
- ClientStats clientStats) {
- return new DefaultBuilder(clientName, clientId, clientBuilder, clientConfig, clientStats);
- }
-
- /**
- * Default Builder for {@link ProxyClient}.
- */
- public static class DefaultBuilder implements Builder {
-
- private final String clientName;
- private final ClientId clientId;
- private final ClientBuilder clientBuilder;
- private final ClientStats clientStats;
-
- private DefaultBuilder(String clientName,
- ClientId clientId,
- ClientBuilder clientBuilder,
- ClientConfig clientConfig,
- ClientStats clientStats) {
- this.clientName = clientName;
- this.clientId = clientId;
- this.clientStats = clientStats;
- // client builder
- ClientBuilder builder = setDefaultSettings(
- null == clientBuilder ? getDefaultClientBuilder(clientConfig) : clientBuilder);
- this.clientBuilder = configureThriftMux(builder, clientId, clientConfig);
- }
-
- @SuppressWarnings("unchecked")
- private ClientBuilder configureThriftMux(ClientBuilder builder,
- ClientId clientId,
- ClientConfig clientConfig) {
- if (clientConfig.getThriftMux()) {
- return builder.stack(ThriftMux.client().withClientId(clientId));
- } else {
- return builder.codec(ThriftClientFramedCodec.apply(Option.apply(clientId)));
- }
- }
-
- private ClientBuilder getDefaultClientBuilder(ClientConfig clientConfig) {
- ClientBuilder builder = ClientBuilder.get()
- .tcpConnectTimeout(Duration.fromMilliseconds(200))
- .connectTimeout(Duration.fromMilliseconds(200))
- .requestTimeout(Duration.fromSeconds(1));
- if (!clientConfig.getThriftMux()) {
- builder = builder.hostConnectionLimit(1);
- }
- return builder;
- }
-
- @SuppressWarnings("unchecked")
- private ClientBuilder setDefaultSettings(ClientBuilder builder) {
- return builder.name(clientName)
- .failFast(false)
- .noFailureAccrual()
- // disable retries on finagle client builder, as there is only one host per finagle client
- // we should throw exception immediately on first failure, so DL client could quickly detect
- // failures and retry other proxies.
- .retries(1)
- .keepAlive(true);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public ProxyClient build(SocketAddress address) {
- Service<ThriftClientRequest, byte[]> client =
- ClientBuilder.safeBuildFactory(
- clientBuilder
- .hosts((InetSocketAddress) address)
- .reportTo(clientStats.getFinagleStatsReceiver(address))
- ).toService();
- DistributedLogService.ServiceIface service =
- new DistributedLogService.ServiceToClient(client, new TBinaryProtocol.Factory());
- return new ProxyClient(address, client, service);
- }
-
- }
-
- private final SocketAddress address;
- private final Service<ThriftClientRequest, byte[]> client;
- private final DistributedLogService.ServiceIface service;
-
- protected ProxyClient(SocketAddress address,
- Service<ThriftClientRequest, byte[]> client,
- DistributedLogService.ServiceIface service) {
- this.address = address;
- this.client = client;
- this.service = service;
- }
-
- public SocketAddress getAddress() {
- return address;
- }
-
- public Service<ThriftClientRequest, byte[]> getClient() {
- return client;
- }
-
- public DistributedLogService.ServiceIface getService() {
- return service;
- }
-
- public Future<BoxedUnit> close() {
- return client.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClientManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClientManager.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClientManager.java
deleted file mode 100644
index c7d56f6..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClientManager.java
+++ /dev/null
@@ -1,362 +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.client.proxy;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.ImmutableMap;
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.stats.ClientStats;
-import com.twitter.distributedlog.client.stats.OpStats;
-import com.twitter.distributedlog.thrift.service.ClientInfo;
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import com.twitter.util.FutureEventListener;
-import java.net.SocketAddress;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timeout;
-import org.jboss.netty.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Manager manages clients (channels) to proxies.
- */
-public class ProxyClientManager implements TimerTask {
-
- private static final Logger logger = LoggerFactory.getLogger(ProxyClientManager.class);
-
- private final ClientConfig clientConfig;
- private final ProxyClient.Builder clientBuilder;
- private final HashedWheelTimer timer;
- private final HostProvider hostProvider;
- private volatile Timeout periodicHandshakeTask;
- private final ConcurrentHashMap<SocketAddress, ProxyClient> address2Services =
- new ConcurrentHashMap<SocketAddress, ProxyClient>();
- private final CopyOnWriteArraySet<ProxyListener> proxyListeners =
- new CopyOnWriteArraySet<ProxyListener>();
- private volatile boolean closed = false;
- private volatile boolean periodicHandshakeEnabled = true;
- private final Stopwatch lastOwnershipSyncStopwatch;
-
- private final OpStats handshakeStats;
-
- public ProxyClientManager(ClientConfig clientConfig,
- ProxyClient.Builder clientBuilder,
- HashedWheelTimer timer,
- HostProvider hostProvider,
- ClientStats clientStats) {
- this.clientConfig = clientConfig;
- this.clientBuilder = clientBuilder;
- this.timer = timer;
- this.hostProvider = hostProvider;
- this.handshakeStats = clientStats.getOpStats("handshake");
- scheduleHandshake();
- this.lastOwnershipSyncStopwatch = Stopwatch.createStarted();
- }
-
- private void scheduleHandshake() {
- if (clientConfig.getPeriodicHandshakeIntervalMs() > 0) {
- periodicHandshakeTask = timer.newTimeout(this,
- clientConfig.getPeriodicHandshakeIntervalMs(), TimeUnit.MILLISECONDS);
- }
- }
-
- void setPeriodicHandshakeEnabled(boolean enabled) {
- this.periodicHandshakeEnabled = enabled;
- }
-
- @Override
- public void run(Timeout timeout) throws Exception {
- if (timeout.isCancelled() || closed) {
- return;
- }
- if (periodicHandshakeEnabled) {
- final boolean syncOwnerships = lastOwnershipSyncStopwatch.elapsed(TimeUnit.MILLISECONDS)
- >= clientConfig.getPeriodicOwnershipSyncIntervalMs();
-
- final Set<SocketAddress> hostsSnapshot = hostProvider.getHosts();
- final AtomicInteger numHosts = new AtomicInteger(hostsSnapshot.size());
- final AtomicInteger numStreams = new AtomicInteger(0);
- final AtomicInteger numSuccesses = new AtomicInteger(0);
- final AtomicInteger numFailures = new AtomicInteger(0);
- final ConcurrentMap<SocketAddress, Integer> streamDistributions =
- new ConcurrentHashMap<SocketAddress, Integer>();
- final Stopwatch stopwatch = Stopwatch.createStarted();
- for (SocketAddress host : hostsSnapshot) {
- final SocketAddress address = host;
- final ProxyClient client = getClient(address);
- handshake(address, client, new FutureEventListener<ServerInfo>() {
- @Override
- public void onSuccess(ServerInfo serverInfo) {
- numStreams.addAndGet(serverInfo.getOwnershipsSize());
- numSuccesses.incrementAndGet();
- notifyHandshakeSuccess(address, client, serverInfo, false, stopwatch);
- if (clientConfig.isHandshakeTracingEnabled()) {
- streamDistributions.putIfAbsent(address, serverInfo.getOwnershipsSize());
- }
- complete();
- }
-
- @Override
- public void onFailure(Throwable cause) {
- numFailures.incrementAndGet();
- notifyHandshakeFailure(address, client, cause, stopwatch);
- complete();
- }
-
- private void complete() {
- if (0 == numHosts.decrementAndGet()) {
- if (syncOwnerships) {
- logger.info("Periodic handshaked with {} hosts : {} streams returned,"
- + " {} hosts succeeded, {} hosts failed",
- new Object[] {
- hostsSnapshot.size(),
- numStreams.get(),
- numSuccesses.get(),
- numFailures.get()});
- if (clientConfig.isHandshakeTracingEnabled()) {
- logger.info("Periodic handshaked stream distribution : {}", streamDistributions);
- }
- }
- }
- }
- }, false, syncOwnerships);
- }
-
- if (syncOwnerships) {
- lastOwnershipSyncStopwatch.reset().start();
- }
- }
- scheduleHandshake();
- }
-
- /**
- * Register a proxy <code>listener</code> on proxy related changes.
- *
- * @param listener
- * proxy listener
- */
- public void registerProxyListener(ProxyListener listener) {
- proxyListeners.add(listener);
- }
-
- private void notifyHandshakeSuccess(SocketAddress address,
- ProxyClient client,
- ServerInfo serverInfo,
- boolean logging,
- Stopwatch stopwatch) {
- if (logging) {
- if (null != serverInfo && serverInfo.isSetOwnerships()) {
- logger.info("Handshaked with {} : {} ownerships returned.",
- address, serverInfo.getOwnerships().size());
- } else {
- logger.info("Handshaked with {} : no ownerships returned", address);
- }
- }
- handshakeStats.completeRequest(address, stopwatch.elapsed(TimeUnit.MICROSECONDS), 1);
- for (ProxyListener listener : proxyListeners) {
- listener.onHandshakeSuccess(address, client, serverInfo);
- }
- }
-
- private void notifyHandshakeFailure(SocketAddress address,
- ProxyClient client,
- Throwable cause,
- Stopwatch stopwatch) {
- handshakeStats.failRequest(address, stopwatch.elapsed(TimeUnit.MICROSECONDS), 1);
- for (ProxyListener listener : proxyListeners) {
- listener.onHandshakeFailure(address, client, cause);
- }
- }
-
- /**
- * Retrieve a client to proxy <code>address</code>.
- *
- * @param address
- * proxy address
- * @return proxy client
- */
- public ProxyClient getClient(final SocketAddress address) {
- ProxyClient sc = address2Services.get(address);
- if (null != sc) {
- return sc;
- }
- return createClient(address);
- }
-
- /**
- * Remove the client to proxy <code>address</code>.
- *
- * @param address
- * proxy address
- */
- public void removeClient(SocketAddress address) {
- ProxyClient sc = address2Services.remove(address);
- if (null != sc) {
- logger.info("Removed host {}.", address);
- sc.close();
- }
- }
-
- /**
- * Remove the client <code>sc</code> to proxy <code>address</code>.
- *
- * @param address
- * proxy address
- * @param sc
- * proxy client
- */
- public void removeClient(SocketAddress address, ProxyClient sc) {
- if (address2Services.remove(address, sc)) {
- logger.info("Remove client {} to host {}.", sc, address);
- sc.close();
- }
- }
-
- /**
- * Create a client to proxy <code>address</code>.
- *
- * @param address
- * proxy address
- * @return proxy client
- */
- public ProxyClient createClient(final SocketAddress address) {
- final ProxyClient sc = clientBuilder.build(address);
- ProxyClient oldSC = address2Services.putIfAbsent(address, sc);
- if (null != oldSC) {
- sc.close();
- return oldSC;
- } else {
- final Stopwatch stopwatch = Stopwatch.createStarted();
- FutureEventListener<ServerInfo> listener = new FutureEventListener<ServerInfo>() {
- @Override
- public void onSuccess(ServerInfo serverInfo) {
- notifyHandshakeSuccess(address, sc, serverInfo, true, stopwatch);
- }
- @Override
- public void onFailure(Throwable cause) {
- notifyHandshakeFailure(address, sc, cause, stopwatch);
- }
- };
- // send a ping messaging after creating connections.
- handshake(address, sc, listener, true, true);
- return sc;
- }
- }
-
- /**
- * Handshake with a given proxy.
- *
- * @param address
- * proxy address
- * @param sc
- * proxy client
- * @param listener
- * listener on handshake result
- */
- private void handshake(SocketAddress address,
- ProxyClient sc,
- FutureEventListener<ServerInfo> listener,
- boolean logging,
- boolean getOwnerships) {
- if (clientConfig.getHandshakeWithClientInfo()) {
- ClientInfo clientInfo = new ClientInfo();
- clientInfo.setGetOwnerships(getOwnerships);
- clientInfo.setStreamNameRegex(clientConfig.getStreamNameRegex());
- if (logging) {
- logger.info("Handshaking with {} : {}", address, clientInfo);
- }
- sc.getService().handshakeWithClientInfo(clientInfo)
- .addEventListener(listener);
- } else {
- if (logging) {
- logger.info("Handshaking with {}", address);
- }
- sc.getService().handshake().addEventListener(listener);
- }
- }
-
- /**
- * Handshake with all proxies.
- *
- * <p>NOTE: this is a synchronous call.
- */
- public void handshake() {
- Set<SocketAddress> hostsSnapshot = hostProvider.getHosts();
- logger.info("Handshaking with {} hosts.", hostsSnapshot.size());
- final CountDownLatch latch = new CountDownLatch(hostsSnapshot.size());
- final Stopwatch stopwatch = Stopwatch.createStarted();
- for (SocketAddress host: hostsSnapshot) {
- final SocketAddress address = host;
- final ProxyClient client = getClient(address);
- handshake(address, client, new FutureEventListener<ServerInfo>() {
- @Override
- public void onSuccess(ServerInfo serverInfo) {
- notifyHandshakeSuccess(address, client, serverInfo, true, stopwatch);
- latch.countDown();
- }
- @Override
- public void onFailure(Throwable cause) {
- notifyHandshakeFailure(address, client, cause, stopwatch);
- latch.countDown();
- }
- }, true, true);
- }
- try {
- latch.await(1, TimeUnit.MINUTES);
- } catch (InterruptedException e) {
- logger.warn("Interrupted on handshaking with servers : ", e);
- }
- }
-
- /**
- * Return number of proxies managed by client manager.
- *
- * @return number of proxies managed by client manager.
- */
- public int getNumProxies() {
- return address2Services.size();
- }
-
- /**
- * Return all clients.
- *
- * @return all clients.
- */
- public Map<SocketAddress, ProxyClient> getAllClients() {
- return ImmutableMap.copyOf(address2Services);
- }
-
- public void close() {
- closed = true;
- Timeout task = periodicHandshakeTask;
- if (null != task) {
- task.cancel();
- }
- for (ProxyClient sc : address2Services.values()) {
- sc.close();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyListener.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyListener.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyListener.java
deleted file mode 100644
index e024825..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyListener.java
+++ /dev/null
@@ -1,50 +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.client.proxy;
-
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import java.net.SocketAddress;
-
-/**
- * Listener on server changes.
- */
-public interface ProxyListener {
- /**
- * When a proxy's server info changed, it would be notified.
- *
- * @param address
- * proxy address
- * @param client
- * proxy client that executes handshaking
- * @param serverInfo
- * proxy's server info
- */
- void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo);
-
- /**
- * Failed to handshake with a proxy.
- *
- * @param address
- * proxy address
- * @param client
- * proxy client
- * @param cause
- * failure reason
- */
- void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/package-info.java
deleted file mode 100644
index dc28c76..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Clients that interact with individual proxies.
- */
-package com.twitter.distributedlog.client.proxy;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/DefaultRegionResolver.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/DefaultRegionResolver.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/DefaultRegionResolver.java
deleted file mode 100644
index ab2fbed..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/DefaultRegionResolver.java
+++ /dev/null
@@ -1,85 +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.client.resolver;
-
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Default implementation of {@link RegionResolver}.
- */
-public class DefaultRegionResolver implements RegionResolver {
-
- private static final String DEFAULT_REGION = "default-region";
-
- private final Map<SocketAddress, String> regionOverrides =
- new HashMap<SocketAddress, String>();
- private final ConcurrentMap<SocketAddress, String> regionMap =
- new ConcurrentHashMap<SocketAddress, String>();
-
- public DefaultRegionResolver() {
- }
-
- public DefaultRegionResolver(Map<SocketAddress, String> regionOverrides) {
- this.regionOverrides.putAll(regionOverrides);
- }
-
- @Override
- public String resolveRegion(SocketAddress address) {
- String region = regionMap.get(address);
- if (null == region) {
- region = doResolveRegion(address);
- regionMap.put(address, region);
- }
- return region;
- }
-
- private String doResolveRegion(SocketAddress address) {
- String region = regionOverrides.get(address);
- if (null != region) {
- return region;
- }
-
- String domainName;
- if (address instanceof InetSocketAddress) {
- InetSocketAddress iAddr = (InetSocketAddress) address;
- domainName = iAddr.getHostName();
- } else {
- domainName = address.toString();
- }
- String[] parts = domainName.split("\\.");
- if (parts.length <= 0) {
- return DEFAULT_REGION;
- }
- String hostName = parts[0];
- String[] labels = hostName.split("-");
- if (labels.length != 4) {
- return DEFAULT_REGION;
- }
- return labels[0];
- }
-
- @Override
- public void removeCachedHost(SocketAddress address) {
- regionMap.remove(address);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/RegionResolver.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/RegionResolver.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/RegionResolver.java
deleted file mode 100644
index eff3aad..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/RegionResolver.java
+++ /dev/null
@@ -1,43 +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.client.resolver;
-
-import java.net.SocketAddress;
-
-/**
- * Resolve address to region.
- */
-public interface RegionResolver {
-
- /**
- * Resolve address to region.
- *
- * @param address
- * socket address
- * @return region
- */
- String resolveRegion(SocketAddress address);
-
- /**
- * Remove cached host.
- *
- * @param address
- * socket address.
- */
- void removeCachedHost(SocketAddress address);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/package-info.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/package-info.java
deleted file mode 100644
index 4bb53a5..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Resolver to resolve network addresses.
- */
-package com.twitter.distributedlog.client.resolver;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ConsistentHashRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ConsistentHashRoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ConsistentHashRoutingService.java
deleted file mode 100644
index 6d1e37e..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ConsistentHashRoutingService.java
+++ /dev/null
@@ -1,500 +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.client.routing;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.finagle.ChannelException;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.Counter;
-import com.twitter.finagle.stats.Gauge;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.util.Function0;
-import java.net.SocketAddress;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.commons.lang3.tuple.Pair;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timeout;
-import org.jboss.netty.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.collection.Seq;
-
-/**
- * Consistent Hashing Based {@link RoutingService}.
- */
-public class ConsistentHashRoutingService extends ServerSetRoutingService {
-
- private static final Logger logger = LoggerFactory.getLogger(ConsistentHashRoutingService.class);
-
- @Deprecated
- public static ConsistentHashRoutingService of(ServerSetWatcher serverSetWatcher, int numReplicas) {
- return new ConsistentHashRoutingService(serverSetWatcher, numReplicas, 300, NullStatsReceiver.get());
- }
-
- /**
- * Builder helper class to build a consistent hash bashed {@link RoutingService}.
- *
- * @return builder to build a consistent hash based {@link RoutingService}.
- */
- public static Builder newBuilder() {
- return new Builder();
- }
-
- /**
- * Builder for building consistent hash based routing service.
- */
- public static class Builder implements RoutingService.Builder {
-
- private ServerSet serverSet;
- private boolean resolveFromName = false;
- private int numReplicas;
- private int blackoutSeconds = 300;
- private StatsReceiver statsReceiver = NullStatsReceiver.get();
-
- private Builder() {}
-
- public Builder serverSet(ServerSet serverSet) {
- this.serverSet = serverSet;
- return this;
- }
-
- public Builder resolveFromName(boolean enabled) {
- this.resolveFromName = enabled;
- return this;
- }
-
- public Builder numReplicas(int numReplicas) {
- this.numReplicas = numReplicas;
- return this;
- }
-
- public Builder blackoutSeconds(int seconds) {
- this.blackoutSeconds = seconds;
- return this;
- }
-
- public Builder statsReceiver(StatsReceiver statsReceiver) {
- this.statsReceiver = statsReceiver;
- return this;
- }
-
- @Override
- public RoutingService build() {
- checkNotNull(serverSet, "No serverset provided.");
- checkNotNull(statsReceiver, "No stats receiver provided.");
- checkArgument(numReplicas > 0, "Invalid number of replicas : " + numReplicas);
- return new ConsistentHashRoutingService(new TwitterServerSetWatcher(serverSet, resolveFromName),
- numReplicas, blackoutSeconds, statsReceiver);
- }
- }
-
- static class ConsistentHash {
- private final HashFunction hashFunction;
- private final int numOfReplicas;
- private final SortedMap<Long, SocketAddress> circle;
-
- // Stats
- protected final Counter hostAddedCounter;
- protected final Counter hostRemovedCounter;
-
- ConsistentHash(HashFunction hashFunction,
- int numOfReplicas,
- StatsReceiver statsReceiver) {
- this.hashFunction = hashFunction;
- this.numOfReplicas = numOfReplicas;
- this.circle = new TreeMap<Long, SocketAddress>();
-
- this.hostAddedCounter = statsReceiver.counter0("adds");
- this.hostRemovedCounter = statsReceiver.counter0("removes");
- }
-
- private String replicaName(int shardId, int replica, String address) {
- if (shardId < 0) {
- shardId = UNKNOWN_SHARD_ID;
- }
-
- StringBuilder sb = new StringBuilder(100);
- sb.append("shard-");
- sb.append(shardId);
- sb.append('-');
- sb.append(replica);
- sb.append('-');
- sb.append(address);
-
- return sb.toString();
- }
-
- private Long replicaHash(int shardId, int replica, String address) {
- return hashFunction.hashUnencodedChars(replicaName(shardId, replica, address)).asLong();
- }
-
- private Long replicaHash(int shardId, int replica, SocketAddress address) {
- return replicaHash(shardId, replica, address.toString());
- }
-
- public synchronized void add(int shardId, SocketAddress address) {
- String addressStr = address.toString();
- for (int i = 0; i < numOfReplicas; i++) {
- Long hash = replicaHash(shardId, i, addressStr);
- circle.put(hash, address);
- }
- hostAddedCounter.incr();
- }
-
- public synchronized void remove(int shardId, SocketAddress address) {
- for (int i = 0; i < numOfReplicas; i++) {
- long hash = replicaHash(shardId, i, address);
- SocketAddress oldAddress = circle.get(hash);
- if (null != oldAddress && oldAddress.equals(address)) {
- circle.remove(hash);
- }
- }
- hostRemovedCounter.incr();
- }
-
- public SocketAddress get(String key, RoutingContext rContext) {
- long hash = hashFunction.hashUnencodedChars(key).asLong();
- return find(hash, rContext);
- }
-
- private synchronized SocketAddress find(long hash, RoutingContext rContext) {
- if (circle.isEmpty()) {
- return null;
- }
-
- Iterator<Map.Entry<Long, SocketAddress>> iterator =
- circle.tailMap(hash).entrySet().iterator();
- while (iterator.hasNext()) {
- Map.Entry<Long, SocketAddress> entry = iterator.next();
- if (!rContext.isTriedHost(entry.getValue())) {
- return entry.getValue();
- }
- }
- // the tail map has been checked
- iterator = circle.headMap(hash).entrySet().iterator();
- while (iterator.hasNext()) {
- Map.Entry<Long, SocketAddress> entry = iterator.next();
- if (!rContext.isTriedHost(entry.getValue())) {
- return entry.getValue();
- }
- }
-
- return null;
- }
-
- private synchronized Pair<Long, SocketAddress> get(long hash) {
- if (circle.isEmpty()) {
- return null;
- }
-
- if (!circle.containsKey(hash)) {
- SortedMap<Long, SocketAddress> tailMap = circle.tailMap(hash);
- hash = tailMap.isEmpty() ? circle.firstKey() : tailMap.firstKey();
- }
- return Pair.of(hash, circle.get(hash));
- }
-
- synchronized void dumpHashRing() {
- for (Map.Entry<Long, SocketAddress> entry : circle.entrySet()) {
- logger.info(entry.getKey() + " : " + entry.getValue());
- }
- }
-
- }
-
- class BlackoutHost implements TimerTask {
- final int shardId;
- final SocketAddress address;
-
- BlackoutHost(int shardId, SocketAddress address) {
- this.shardId = shardId;
- this.address = address;
- numBlackoutHosts.incrementAndGet();
- }
-
- @Override
- public void run(Timeout timeout) throws Exception {
- numBlackoutHosts.decrementAndGet();
- if (!timeout.isExpired()) {
- return;
- }
- Set<SocketAddress> removedList = new HashSet<SocketAddress>();
- boolean joined;
- // add the shard back
- synchronized (shardId2Address) {
- SocketAddress curHost = shardId2Address.get(shardId);
- if (null != curHost) {
- // there is already new shard joint, so drop the host.
- logger.info("Blackout Shard {} ({}) was already replaced by {} permanently.",
- new Object[] { shardId, address, curHost });
- joined = false;
- } else {
- join(shardId, address, removedList);
- joined = true;
- }
- }
- if (joined) {
- for (RoutingListener listener : listeners) {
- listener.onServerJoin(address);
- }
- } else {
- for (RoutingListener listener : listeners) {
- listener.onServerLeft(address);
- }
- }
- }
- }
-
- protected final HashedWheelTimer hashedWheelTimer;
- protected final HashFunction hashFunction = Hashing.md5();
- protected final ConsistentHash circle;
- protected final Map<Integer, SocketAddress> shardId2Address =
- new HashMap<Integer, SocketAddress>();
- protected final Map<SocketAddress, Integer> address2ShardId =
- new HashMap<SocketAddress, Integer>();
-
- // blackout period
- protected final int blackoutSeconds;
-
- // stats
- protected final StatsReceiver statsReceiver;
- protected final AtomicInteger numBlackoutHosts;
- protected final Gauge numBlackoutHostsGauge;
- protected final Gauge numHostsGauge;
-
- private static final int UNKNOWN_SHARD_ID = -1;
-
- ConsistentHashRoutingService(ServerSetWatcher serverSetWatcher,
- int numReplicas,
- int blackoutSeconds,
- StatsReceiver statsReceiver) {
- super(serverSetWatcher);
- this.circle = new ConsistentHash(hashFunction, numReplicas, statsReceiver.scope("ring"));
- this.hashedWheelTimer = new HashedWheelTimer(new ThreadFactoryBuilder()
- .setNameFormat("ConsistentHashRoutingService-Timer-%d").build());
- this.blackoutSeconds = blackoutSeconds;
- // stats
- this.statsReceiver = statsReceiver;
- this.numBlackoutHosts = new AtomicInteger(0);
- this.numBlackoutHostsGauge = this.statsReceiver.addGauge(gaugeName("num_blackout_hosts"),
- new Function0<Object>() {
- @Override
- public Object apply() {
- return (float) numBlackoutHosts.get();
- }
- });
- this.numHostsGauge = this.statsReceiver.addGauge(gaugeName("num_hosts"),
- new Function0<Object>() {
- @Override
- public Object apply() {
- return (float) address2ShardId.size();
- }
- });
- }
-
- private static Seq<String> gaugeName(String name) {
- return scala.collection.JavaConversions.asScalaBuffer(Arrays.asList(name)).toList();
- }
-
- @Override
- public void startService() {
- super.startService();
- this.hashedWheelTimer.start();
- }
-
- @Override
- public void stopService() {
- this.hashedWheelTimer.stop();
- super.stopService();
- }
-
- @Override
- public Set<SocketAddress> getHosts() {
- synchronized (shardId2Address) {
- return ImmutableSet.copyOf(address2ShardId.keySet());
- }
- }
-
- @Override
- public SocketAddress getHost(String key, RoutingContext rContext)
- throws NoBrokersAvailableException {
- SocketAddress host = circle.get(key, rContext);
- if (null != host) {
- return host;
- }
- throw new NoBrokersAvailableException("No host found for " + key + ", routing context : " + rContext);
- }
-
- @Override
- public void removeHost(SocketAddress host, Throwable reason) {
- removeHostInternal(host, Optional.of(reason));
- }
-
- private void removeHostInternal(SocketAddress host, Optional<Throwable> reason) {
- synchronized (shardId2Address) {
- Integer shardId = address2ShardId.remove(host);
- if (null != shardId) {
- SocketAddress curHost = shardId2Address.get(shardId);
- if (null != curHost && curHost.equals(host)) {
- shardId2Address.remove(shardId);
- }
- circle.remove(shardId, host);
- if (reason.isPresent()) {
- if (reason.get() instanceof ChannelException) {
- logger.info("Shard {} ({}) left due to ChannelException, black it out for {} seconds"
- + " (message = {})",
- new Object[] { shardId, host, blackoutSeconds, reason.get().toString() });
- BlackoutHost blackoutHost = new BlackoutHost(shardId, host);
- hashedWheelTimer.newTimeout(blackoutHost, blackoutSeconds, TimeUnit.SECONDS);
- } else {
- logger.info("Shard {} ({}) left due to exception {}",
- new Object[] { shardId, host, reason.get().toString() });
- }
- } else {
- logger.info("Shard {} ({}) left after server set change",
- shardId, host);
- }
- } else if (reason.isPresent()) {
- logger.info("Node {} left due to exception {}", host, reason.get().toString());
- } else {
- logger.info("Node {} left after server set change", host);
- }
- }
- }
-
- /**
- * The caller should synchronize on <i>shardId2Address</i>.
- * @param shardId
- * Shard id of new host joined.
- * @param newHost
- * New host joined.
- * @param removedList
- * Old hosts to remove
- */
- private void join(int shardId, SocketAddress newHost, Set<SocketAddress> removedList) {
- SocketAddress oldHost = shardId2Address.put(shardId, newHost);
- if (null != oldHost) {
- // remove the old host only when a new shard is kicked in to replace it.
- address2ShardId.remove(oldHost);
- circle.remove(shardId, oldHost);
- removedList.add(oldHost);
- logger.info("Shard {} ({}) left permanently.", shardId, oldHost);
- }
- address2ShardId.put(newHost, shardId);
- circle.add(shardId, newHost);
- logger.info("Shard {} ({}) joined to replace ({}).",
- new Object[] { shardId, newHost, oldHost });
- }
-
- @Override
- protected synchronized void performServerSetChange(ImmutableSet<DLSocketAddress> serviceInstances) {
- Set<SocketAddress> joinedList = new HashSet<SocketAddress>();
- Set<SocketAddress> removedList = new HashSet<SocketAddress>();
-
- Map<Integer, SocketAddress> newMap = new HashMap<Integer, SocketAddress>();
- synchronized (shardId2Address) {
- for (DLSocketAddress serviceInstance : serviceInstances) {
- if (serviceInstance.getShard() >= 0) {
- newMap.put(serviceInstance.getShard(), serviceInstance.getSocketAddress());
- } else {
- Integer shard = address2ShardId.get(serviceInstance.getSocketAddress());
- if (null == shard) {
- // Assign a random negative shardId
- int shardId;
- do {
- shardId = Math.min(-1 , (int) (Math.random() * Integer.MIN_VALUE));
- } while (null != shardId2Address.get(shardId));
- shard = shardId;
- }
- newMap.put(shard, serviceInstance.getSocketAddress());
- }
- }
- }
-
- Map<Integer, SocketAddress> left;
- synchronized (shardId2Address) {
- MapDifference<Integer, SocketAddress> difference =
- Maps.difference(shardId2Address, newMap);
- left = difference.entriesOnlyOnLeft();
- for (Map.Entry<Integer, SocketAddress> shardEntry : left.entrySet()) {
- int shard = shardEntry.getKey();
- if (shard >= 0) {
- SocketAddress host = shardId2Address.get(shard);
- if (null != host) {
- // we don't remove those hosts that just disappered on serverset proactively,
- // since it might be just because serverset become flaky
- // address2ShardId.remove(host);
- // circle.remove(shard, host);
- logger.info("Shard {} ({}) left temporarily.", shard, host);
- }
- } else {
- // shard id is negative - they are resolved from finagle name, which instances don't have shard id
- // in this case, if they are removed from serverset, we removed them directly
- SocketAddress host = shardEntry.getValue();
- if (null != host) {
- removeHostInternal(host, Optional.<Throwable>absent());
- removedList.add(host);
- }
- }
- }
- // we need to find if any shards are replacing old shards
- for (Map.Entry<Integer, SocketAddress> shard : newMap.entrySet()) {
- SocketAddress oldHost = shardId2Address.get(shard.getKey());
- SocketAddress newHost = shard.getValue();
- if (!newHost.equals(oldHost)) {
- join(shard.getKey(), newHost, removedList);
- joinedList.add(newHost);
- }
- }
- }
-
- for (SocketAddress addr : removedList) {
- for (RoutingListener listener : listeners) {
- listener.onServerLeft(addr);
- }
- }
-
- for (SocketAddress addr : joinedList) {
- for (RoutingListener listener : listeners) {
- listener.onServerJoin(addr);
- }
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/NameServerSet.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/NameServerSet.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/NameServerSet.java
deleted file mode 100644
index eeba4ac..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/NameServerSet.java
+++ /dev/null
@@ -1,263 +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.client.routing;
-
-import com.google.common.collect.ImmutableSet;
-import com.twitter.common.base.Command;
-import com.twitter.common.base.Commands;
-import com.twitter.common.zookeeper.Group;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.finagle.Addr;
-import com.twitter.finagle.Address;
-import com.twitter.finagle.Name;
-import com.twitter.finagle.Resolver$;
-import com.twitter.thrift.Endpoint;
-import com.twitter.thrift.ServiceInstance;
-import com.twitter.thrift.Status;
-import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-/**
- * Finagle Name based {@link ServerSet} implementation.
- */
-class NameServerSet implements ServerSet {
-
- private static final Logger logger = LoggerFactory.getLogger(NameServerSet.class);
-
- private volatile Set<HostChangeMonitor<ServiceInstance>> watchers =
- new HashSet<HostChangeMonitor<ServiceInstance>>();
- private volatile ImmutableSet<ServiceInstance> hostSet = ImmutableSet.of();
- private AtomicBoolean resolutionPending = new AtomicBoolean(true);
-
- public NameServerSet(String nameStr) {
- Name name;
- try {
- name = Resolver$.MODULE$.eval(nameStr);
- } catch (Exception exc) {
- logger.error("Exception in Resolver.eval for name {}", nameStr, exc);
- // Since this is called from various places that dont handle specific exceptions,
- // we have no option than to throw a runtime exception to halt the control flow
- // This should only happen in case of incorrect configuration. Having a log message
- // would help identify the problem during tests
- throw new RuntimeException(exc);
- }
- initialize(name);
- }
-
- public NameServerSet(Name name) {
- initialize(name);
- }
-
- private void initialize(Name name) {
- if (name instanceof TestName) {
- ((TestName) name).changes(new AbstractFunction1<Addr, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Addr varAddr) {
- return NameServerSet.this.respondToChanges(varAddr);
- }
- });
- } else if (name instanceof Name.Bound) {
- ((Name.Bound) name).addr().changes().respond(new AbstractFunction1<Addr, BoxedUnit>() {
- @Override
- public BoxedUnit apply(Addr varAddr) {
- return NameServerSet.this.respondToChanges(varAddr);
- }
- });
- } else {
- logger.error("NameServerSet only supports Name.Bound. While the resolved name {} was {}",
- name, name.getClass());
- throw new UnsupportedOperationException("NameServerSet only supports Name.Bound");
- }
- }
-
- private ServiceInstance endpointAddressToServiceInstance(Address endpointAddress) {
- if (endpointAddress instanceof Address.Inet) {
- InetSocketAddress inetSocketAddress = ((Address.Inet) endpointAddress).addr();
- Endpoint endpoint = new Endpoint(inetSocketAddress.getHostString(), inetSocketAddress.getPort());
- HashMap<String, Endpoint> map = new HashMap<String, Endpoint>();
- map.put("thrift", endpoint);
- return new ServiceInstance(
- endpoint,
- map,
- Status.ALIVE);
- } else {
- logger.error("We expect InetSocketAddress while the resolved address {} was {}",
- endpointAddress, endpointAddress.getClass());
- throw new UnsupportedOperationException("invalid endpoint address: " + endpointAddress);
- }
- }
-
-
- private BoxedUnit respondToChanges(Addr addr) {
- ImmutableSet<ServiceInstance> oldHostSet = ImmutableSet.copyOf(hostSet);
-
- ImmutableSet<ServiceInstance> newHostSet = oldHostSet;
-
- if (addr instanceof Addr.Bound) {
- scala.collection.immutable.Set<Address> endpointAddresses = ((Addr.Bound) addr).addrs();
- scala.collection.Iterator<Address> endpointAddressesIterator = endpointAddresses.toIterator();
- HashSet<ServiceInstance> serviceInstances = new HashSet<ServiceInstance>();
- while (endpointAddressesIterator.hasNext()) {
- serviceInstances.add(endpointAddressToServiceInstance(endpointAddressesIterator.next()));
- }
- newHostSet = ImmutableSet.copyOf(serviceInstances);
-
- } else if (addr instanceof Addr.Failed) {
- logger.error("Name resolution failed", ((Addr.Failed) addr).cause());
- newHostSet = ImmutableSet.of();
- } else if (addr.toString().equals("Pending")) {
- logger.info("Name resolution pending");
- newHostSet = oldHostSet;
- } else if (addr.toString().equals("Neg")) {
- newHostSet = ImmutableSet.of();
- } else {
- logger.error("Invalid Addr type: {}", addr.getClass().getName());
- throw new UnsupportedOperationException("Invalid Addr type:" + addr.getClass().getName());
- }
-
- // Reference comparison is valid as the sets are immutable
- if (oldHostSet != newHostSet) {
- logger.info("NameServerSet updated: {} -> {}", hostSetToString(oldHostSet), hostSetToString(newHostSet));
- resolutionPending.set(false);
- hostSet = newHostSet;
- synchronized (watchers) {
- for (HostChangeMonitor<ServiceInstance> watcher: watchers) {
- watcher.onChange(newHostSet);
- }
- }
-
- }
-
- return BoxedUnit.UNIT;
- }
-
-
- private String hostSetToString(ImmutableSet<ServiceInstance> hostSet) {
- StringBuilder result = new StringBuilder();
- result.append("(");
- for (ServiceInstance serviceInstance : hostSet) {
- Endpoint endpoint = serviceInstance.getServiceEndpoint();
- result.append(String.format(" %s:%d", endpoint.getHost(), endpoint.getPort()));
- }
- result.append(" )");
-
- return result.toString();
- }
-
-
- /**
- * Attempts to join a server set for this logical service group.
- *
- * @param endpoint the primary service endpoint
- * @param additionalEndpoints and additional endpoints keyed by their logical name
- * @param status the current service status
- * @return an EndpointStatus object that allows the endpoint to adjust its status
- * @throws Group.JoinException if there was a problem joining the server set
- * @throws InterruptedException if interrupted while waiting to join the server set
- * @deprecated The status field is deprecated. Please use {@link #join(java.net.InetSocketAddress, java.util.Map)}
- */
- @Override
- public EndpointStatus join(InetSocketAddress endpoint,
- Map<String, InetSocketAddress> additionalEndpoints,
- Status status)
- throws Group.JoinException, InterruptedException {
- throw new UnsupportedOperationException("NameServerSet does not support join");
- }
-
- /**
- * Attempts to join a server set for this logical service group.
- *
- * @param endpoint the primary service endpoint
- * @param additionalEndpoints and additional endpoints keyed by their logical name
- * @return an EndpointStatus object that allows the endpoint to adjust its status
- * @throws Group.JoinException if there was a problem joining the server set
- * @throws InterruptedException if interrupted while waiting to join the server set
- */
- @Override
- public EndpointStatus join(InetSocketAddress endpoint, Map<String, InetSocketAddress> additionalEndpoints)
- throws Group.JoinException, InterruptedException {
- throw new UnsupportedOperationException("NameServerSet does not support join");
- }
-
- /**
- * Attempts to join a server set for this logical service group.
- *
- * @param endpoint the primary service endpoint
- * @param additionalEndpoints and additional endpoints keyed by their logical name
- * @param shardId Unique shard identifier for this member of the service.
- * @return an EndpointStatus object that allows the endpoint to adjust its status
- * @throws Group.JoinException if there was a problem joining the server set
- * @throws InterruptedException if interrupted while waiting to join the server set
- */
- @Override
- public EndpointStatus join(InetSocketAddress endpoint,
- Map<String, InetSocketAddress> additionalEndpoints,
- int shardId)
- throws Group.JoinException, InterruptedException {
- throw new UnsupportedOperationException("NameServerSet does not support join");
- }
-
- /**
- * Registers a monitor to receive change notices for this server set as long as this jvm process
- * is alive. Blocks until the initial server set can be gathered and delivered to the monitor.
- * The monitor will be notified if the membership set or parameters of existing members have
- * changed.
- *
- * @param monitor the server set monitor to call back when the host set changes
- * @throws com.twitter.common.net.pool.DynamicHostSet.MonitorException if there is a problem monitoring the host set
- * @deprecated Deprecated in favor of {@link #watch(com.twitter.common.net.pool.DynamicHostSet.HostChangeMonitor)}
- */
- @Deprecated
- @Override
- public void monitor(HostChangeMonitor<ServiceInstance> monitor) throws MonitorException {
- throw new UnsupportedOperationException("NameServerSet does not support monitor");
- }
-
- /**
- * Registers a monitor to receive change notices for this server set as long as this jvm process
- * is alive. Blocks until the initial server set can be gathered and delivered to the monitor.
- * The monitor will be notified if the membership set or parameters of existing members have
- * changed.
- *
- * @param monitor the server set monitor to call back when the host set changes
- * @return A command which, when executed, will stop monitoring the host set.
- * @throws com.twitter.common.net.pool.DynamicHostSet.MonitorException if there is a problem monitoring the host set
- */
- @Override
- public Command watch(HostChangeMonitor<ServiceInstance> monitor) throws MonitorException {
- // First add the monitor to the watchers so that it does not miss any changes and invoke
- // the onChange method
- synchronized (watchers) {
- watchers.add(monitor);
- }
-
- if (resolutionPending.compareAndSet(false, false)) {
- monitor.onChange(hostSet);
- }
-
- return Commands.NOOP; // Return value is not used
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RegionsRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RegionsRoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RegionsRoutingService.java
deleted file mode 100644
index 4714270..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RegionsRoutingService.java
+++ /dev/null
@@ -1,192 +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.client.routing;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.Set;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Chain multiple routing services.
- */
-public class RegionsRoutingService implements RoutingService {
-
- private static final Logger logger = LoggerFactory.getLogger(RegionsRoutingService.class);
-
- /**
- * Create a multiple regions routing services based on a list of region routing {@code services}.
- *
- * <p>It is deprecated. Please use {@link Builder} to build multiple regions routing service.
- *
- * @param regionResolver region resolver
- * @param services a list of region routing services.
- * @return multiple regions routing service
- * @see Builder
- */
- @Deprecated
- public static RegionsRoutingService of(RegionResolver regionResolver,
- RoutingService...services) {
- return new RegionsRoutingService(regionResolver, services);
- }
-
- /**
- * Create a builder to build a multiple-regions routing service.
- *
- * @return builder to build a multiple-regions routing service.
- */
- public static Builder newBuilder() {
- return new Builder();
- }
-
- /**
- * Builder to build a multiple-regions routing service.
- */
- public static class Builder implements RoutingService.Builder {
-
- private RegionResolver resolver;
- private RoutingService.Builder[] routingServiceBuilders;
- private StatsReceiver statsReceiver = NullStatsReceiver.get();
-
- private Builder() {}
-
- public Builder routingServiceBuilders(RoutingService.Builder...builders) {
- this.routingServiceBuilders = builders;
- return this;
- }
-
- public Builder resolver(RegionResolver regionResolver) {
- this.resolver = regionResolver;
- return this;
- }
-
- @Override
- public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
- this.statsReceiver = statsReceiver;
- return this;
- }
-
- @Override
- public RegionsRoutingService build() {
- checkNotNull(routingServiceBuilders, "No routing service builder provided.");
- checkNotNull(resolver, "No region resolver provided.");
- checkNotNull(statsReceiver, "No stats receiver provided");
- RoutingService[] services = new RoutingService[routingServiceBuilders.length];
- for (int i = 0; i < services.length; i++) {
- String statsScope;
- if (0 == i) {
- statsScope = "local";
- } else {
- statsScope = "remote_" + i;
- }
- services[i] = routingServiceBuilders[i]
- .statsReceiver(statsReceiver.scope(statsScope))
- .build();
- }
- return new RegionsRoutingService(resolver, services);
- }
- }
-
- protected final RegionResolver regionResolver;
- protected final RoutingService[] routingServices;
-
- private RegionsRoutingService(RegionResolver resolver,
- RoutingService[] routingServices) {
- this.regionResolver = resolver;
- this.routingServices = routingServices;
- }
-
- @Override
- public Set<SocketAddress> getHosts() {
- Set<SocketAddress> hosts = Sets.newHashSet();
- for (RoutingService rs : routingServices) {
- hosts.addAll(rs.getHosts());
- }
- return hosts;
- }
-
- @Override
- public void startService() {
- for (RoutingService service : routingServices) {
- service.startService();
- }
- logger.info("Regions Routing Service Started");
- }
-
- @Override
- public void stopService() {
- for (RoutingService service : routingServices) {
- service.stopService();
- }
- logger.info("Regions Routing Service Stopped");
- }
-
- @Override
- public RoutingService registerListener(RoutingListener listener) {
- for (RoutingService service : routingServices) {
- service.registerListener(listener);
- }
- return this;
- }
-
- @Override
- public RoutingService unregisterListener(RoutingListener listener) {
- for (RoutingService service : routingServices) {
- service.registerListener(listener);
- }
- return this;
- }
-
- @Override
- public SocketAddress getHost(String key, RoutingContext routingContext)
- throws NoBrokersAvailableException {
- for (RoutingService service : routingServices) {
- try {
- SocketAddress addr = service.getHost(key, routingContext);
- if (routingContext.hasUnavailableRegions()) {
- // current region is unavailable
- String region = regionResolver.resolveRegion(addr);
- if (routingContext.isUnavailableRegion(region)) {
- continue;
- }
- }
- if (!routingContext.isTriedHost(addr)) {
- return addr;
- }
- } catch (NoBrokersAvailableException nbae) {
- // if there isn't broker available in current service, try next service.
- logger.debug("No brokers available in region {} : ", service, nbae);
- }
- }
- throw new NoBrokersAvailableException("No host found for " + key + ", routing context : " + routingContext);
- }
-
- @Override
- public void removeHost(SocketAddress address, Throwable reason) {
- for (RoutingService service : routingServices) {
- service.removeHost(address, reason);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingService.java
deleted file mode 100644
index 56446c1..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingService.java
+++ /dev/null
@@ -1,206 +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.client.routing;
-
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Routing Service provides mechanism how to route requests.
- */
-public interface RoutingService {
-
- /**
- * Builder to build routing service.
- */
- interface Builder {
-
- /**
- * Build routing service with stats receiver.
- *
- * @param statsReceiver
- * stats receiver
- * @return built routing service
- */
- Builder statsReceiver(StatsReceiver statsReceiver);
-
- /**
- * Build the routing service.
- *
- * @return built routing service
- */
- RoutingService build();
-
- }
-
- /**
- * Listener for server changes on routing service.
- */
- interface RoutingListener {
- /**
- * Trigger when server left.
- *
- * @param address left server.
- */
- void onServerLeft(SocketAddress address);
-
- /**
- * Trigger when server joint.
- *
- * @param address joint server.
- */
- void onServerJoin(SocketAddress address);
- }
-
- /**
- * Routing Context of a request.
- */
- class RoutingContext {
-
- public static RoutingContext of(RegionResolver resolver) {
- return new RoutingContext(resolver);
- }
-
- final RegionResolver regionResolver;
- final Map<SocketAddress, StatusCode> triedHosts;
- final Set<String> unavailableRegions;
-
- private RoutingContext(RegionResolver regionResolver) {
- this.regionResolver = regionResolver;
- this.triedHosts = new HashMap<SocketAddress, StatusCode>();
- this.unavailableRegions = new HashSet<String>();
- }
-
- @Override
- public synchronized String toString() {
- return "(tried hosts=" + triedHosts + ")";
- }
-
- /**
- * Add tried host to routing context.
- *
- * @param socketAddress
- * socket address of tried host.
- * @param code
- * status code returned from tried host.
- * @return routing context.
- */
- public synchronized RoutingContext addTriedHost(SocketAddress socketAddress, StatusCode code) {
- this.triedHosts.put(socketAddress, code);
- if (StatusCode.REGION_UNAVAILABLE == code) {
- unavailableRegions.add(regionResolver.resolveRegion(socketAddress));
- }
- return this;
- }
-
- /**
- * Is the host <i>address</i> already tried.
- *
- * @param address
- * socket address to check
- * @return true if the address is already tried, otherwise false.
- */
- public synchronized boolean isTriedHost(SocketAddress address) {
- return this.triedHosts.containsKey(address);
- }
-
- /**
- * Whether encountered unavailable regions.
- *
- * @return true if encountered unavailable regions, otherwise false.
- */
- public synchronized boolean hasUnavailableRegions() {
- return !unavailableRegions.isEmpty();
- }
-
- /**
- * Whether the <i>region</i> is unavailable.
- *
- * @param region
- * region
- * @return true if the region is unavailable, otherwise false.
- */
- public synchronized boolean isUnavailableRegion(String region) {
- return unavailableRegions.contains(region);
- }
-
- }
-
- /**
- * Start routing service.
- */
- void startService();
-
- /**
- * Stop routing service.
- */
- void stopService();
-
- /**
- * Register routing listener.
- *
- * @param listener routing listener.
- * @return routing service.
- */
- RoutingService registerListener(RoutingListener listener);
-
- /**
- * Unregister routing listener.
- *
- * @param listener routing listener.
- * @return routing service.
- */
- RoutingService unregisterListener(RoutingListener listener);
-
- /**
- * Get all the hosts that available in routing service.
- *
- * @return all the hosts
- */
- Set<SocketAddress> getHosts();
-
- /**
- * Get the host to route the request by <i>key</i>.
- *
- * @param key
- * key to route the request.
- * @param rContext
- * routing context.
- * @return host to route the request
- * @throws NoBrokersAvailableException
- */
- SocketAddress getHost(String key, RoutingContext rContext)
- throws NoBrokersAvailableException;
-
- /**
- * Remove the host <i>address</i> for a specific <i>reason</i>.
- *
- * @param address
- * host address to remove
- * @param reason
- * reason to remove the host
- */
- void removeHost(SocketAddress address, Throwable reason);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingServiceProvider.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingServiceProvider.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingServiceProvider.java
deleted file mode 100644
index 22cd222..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingServiceProvider.java
+++ /dev/null
@@ -1,39 +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.client.routing;
-
-import com.twitter.finagle.stats.StatsReceiver;
-
-class RoutingServiceProvider implements RoutingService.Builder {
-
- final RoutingService routingService;
-
- RoutingServiceProvider(RoutingService routingService) {
- this.routingService = routingService;
- }
-
- @Override
- public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
- return this;
- }
-
- @Override
- public RoutingService build() {
- return routingService;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingUtils.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingUtils.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingUtils.java
deleted file mode 100644
index 2302e18..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingUtils.java
+++ /dev/null
@@ -1,88 +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.client.routing;
-
-import com.twitter.common.zookeeper.ServerSet;
-import java.net.SocketAddress;
-
-/**
- * Utils for routing services.
- */
-public class RoutingUtils {
-
- private static final int NUM_CONSISTENT_HASH_REPLICAS = 997;
-
- /**
- * Building routing service from <code>finagleNameStr</code>.
- *
- * @param finagleNameStr
- * finagle name str of a service
- * @return routing service builder
- */
- public static RoutingService.Builder buildRoutingService(String finagleNameStr) {
- if (!finagleNameStr.startsWith("serverset!")
- && !finagleNameStr.startsWith("inet!")
- && !finagleNameStr.startsWith("zk!")) {
- // We only support serverset based names at the moment
- throw new UnsupportedOperationException("Finagle Name format not supported for name: " + finagleNameStr);
- }
- return buildRoutingService(new NameServerSet(finagleNameStr), true);
- }
-
- /**
- * Building routing service from <code>serverSet</code>.
- *
- * @param serverSet
- * server set of a service
- * @return routing service builder
- */
- public static RoutingService.Builder buildRoutingService(ServerSet serverSet) {
- return buildRoutingService(serverSet, false);
- }
-
- /**
- * Building routing service from <code>address</code>.
- *
- * @param address
- * host to route the requests
- * @return routing service builder
- */
- public static RoutingService.Builder buildRoutingService(SocketAddress address) {
- return SingleHostRoutingService.newBuilder().address(address);
- }
-
- /**
- * Build routing service builder of a routing service <code>routingService</code>.
- *
- * @param routingService
- * routing service to provide
- * @return routing service builder
- */
- public static RoutingService.Builder buildRoutingService(RoutingService routingService) {
- return new RoutingServiceProvider(routingService);
- }
-
- private static RoutingService.Builder buildRoutingService(ServerSet serverSet,
- boolean resolveFromName) {
- return ConsistentHashRoutingService.newBuilder()
- .serverSet(serverSet)
- .resolveFromName(resolveFromName)
- .numReplicas(NUM_CONSISTENT_HASH_REPLICAS);
- }
-
-}
[40/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/ownership/TestOwnershipCache.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/ownership/TestOwnershipCache.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/ownership/TestOwnershipCache.java
deleted file mode 100644
index c0e077b..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/ownership/TestOwnershipCache.java
+++ /dev/null
@@ -1,207 +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.client.ownership;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.Map;
-import java.util.Set;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test Case for Ownership Cache.
- */
-public class TestOwnershipCache {
-
- @Rule
- public TestName runtime = new TestName();
-
- private static OwnershipCache createOwnershipCache() {
- ClientConfig clientConfig = new ClientConfig();
- return new OwnershipCache(clientConfig, null,
- NullStatsReceiver.get(), NullStatsReceiver.get());
- }
-
- private static SocketAddress createSocketAddress(int port) {
- return new InetSocketAddress("127.0.0.1", port);
- }
-
- @Test(timeout = 60000)
- public void testUpdateOwner() {
- OwnershipCache cache = createOwnershipCache();
- SocketAddress addr = createSocketAddress(1000);
- String stream = runtime.getMethodName();
-
- assertTrue("Should successfully update owner if no owner exists before",
- cache.updateOwner(stream, addr));
- assertEquals("Owner should be " + addr + " for stream " + stream,
- addr, cache.getOwner(stream));
- assertTrue("Should successfully update owner if old owner is same",
- cache.updateOwner(stream, addr));
- assertEquals("Owner should be " + addr + " for stream " + stream,
- addr, cache.getOwner(stream));
- }
-
- @Test(timeout = 60000)
- public void testRemoveOwnerFromStream() {
- OwnershipCache cache = createOwnershipCache();
- int initialPort = 2000;
- int numProxies = 2;
- int numStreamsPerProxy = 2;
- for (int i = 0; i < numProxies; i++) {
- SocketAddress addr = createSocketAddress(initialPort + i);
- for (int j = 0; j < numStreamsPerProxy; j++) {
- String stream = runtime.getMethodName() + "_" + i + "_" + j;
- cache.updateOwner(stream, addr);
- }
- }
- Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
- assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
- numProxies * numStreamsPerProxy, ownershipMap.size());
- Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
- assertEquals("There should be " + numProxies + " proxies cached",
- numProxies, ownershipDistribution.size());
-
- String stream = runtime.getMethodName() + "_0_0";
- SocketAddress owner = createSocketAddress(initialPort);
-
- // remove non-existent mapping won't change anything
- SocketAddress nonExistentAddr = createSocketAddress(initialPort + 999);
- cache.removeOwnerFromStream(stream, nonExistentAddr, "remove-non-existent-addr");
- assertEquals("Owner " + owner + " should not be removed",
- owner, cache.getOwner(stream));
- ownershipMap = cache.getStreamOwnerMapping();
- assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
- numProxies * numStreamsPerProxy, ownershipMap.size());
-
- // remove existent mapping should remove ownership mapping
- cache.removeOwnerFromStream(stream, owner, "remove-owner");
- assertNull("Owner " + owner + " should be removed", cache.getOwner(stream));
- ownershipMap = cache.getStreamOwnerMapping();
- assertEquals("There should be " + (numProxies * numStreamsPerProxy - 1) + " entries left in cache",
- numProxies * numStreamsPerProxy - 1, ownershipMap.size());
- ownershipDistribution = cache.getStreamOwnershipDistribution();
- assertEquals("There should still be " + numProxies + " proxies cached",
- numProxies, ownershipDistribution.size());
- Set<String> ownedStreams = ownershipDistribution.get(owner);
- assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned for " + owner,
- numStreamsPerProxy - 1, ownedStreams.size());
- assertFalse("Stream " + stream + " should not be owned by " + owner,
- ownedStreams.contains(stream));
- }
-
- @Test(timeout = 60000)
- public void testRemoveAllStreamsFromOwner() {
- OwnershipCache cache = createOwnershipCache();
- int initialPort = 2000;
- int numProxies = 2;
- int numStreamsPerProxy = 2;
- for (int i = 0; i < numProxies; i++) {
- SocketAddress addr = createSocketAddress(initialPort + i);
- for (int j = 0; j < numStreamsPerProxy; j++) {
- String stream = runtime.getMethodName() + "_" + i + "_" + j;
- cache.updateOwner(stream, addr);
- }
- }
- Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
- assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
- numProxies * numStreamsPerProxy, ownershipMap.size());
- Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
- assertEquals("There should be " + numProxies + " proxies cached",
- numProxies, ownershipDistribution.size());
-
- SocketAddress owner = createSocketAddress(initialPort);
-
- // remove non-existent host won't change anything
- SocketAddress nonExistentAddr = createSocketAddress(initialPort + 999);
- cache.removeAllStreamsFromOwner(nonExistentAddr);
- ownershipMap = cache.getStreamOwnerMapping();
- assertEquals("There should still be " + (numProxies * numStreamsPerProxy) + " entries in cache",
- numProxies * numStreamsPerProxy, ownershipMap.size());
- ownershipDistribution = cache.getStreamOwnershipDistribution();
- assertEquals("There should still be " + numProxies + " proxies cached",
- numProxies, ownershipDistribution.size());
-
- // remove existent host should remove ownership mapping
- cache.removeAllStreamsFromOwner(owner);
- ownershipMap = cache.getStreamOwnerMapping();
- assertEquals("There should be " + ((numProxies - 1) * numStreamsPerProxy) + " entries left in cache",
- (numProxies - 1) * numStreamsPerProxy, ownershipMap.size());
- ownershipDistribution = cache.getStreamOwnershipDistribution();
- assertEquals("There should be " + (numProxies - 1) + " proxies cached",
- numProxies - 1, ownershipDistribution.size());
- assertFalse("Host " + owner + " should not be cached",
- ownershipDistribution.containsKey(owner));
- }
-
- @Test(timeout = 60000)
- public void testReplaceOwner() {
- OwnershipCache cache = createOwnershipCache();
- int initialPort = 2000;
- int numProxies = 2;
- int numStreamsPerProxy = 2;
- for (int i = 0; i < numProxies; i++) {
- SocketAddress addr = createSocketAddress(initialPort + i);
- for (int j = 0; j < numStreamsPerProxy; j++) {
- String stream = runtime.getMethodName() + "_" + i + "_" + j;
- cache.updateOwner(stream, addr);
- }
- }
- Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
- assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
- numProxies * numStreamsPerProxy, ownershipMap.size());
- Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
- assertEquals("There should be " + numProxies + " proxies cached",
- numProxies, ownershipDistribution.size());
-
- String stream = runtime.getMethodName() + "_0_0";
- SocketAddress oldOwner = createSocketAddress(initialPort);
- SocketAddress newOwner = createSocketAddress(initialPort + 999);
-
- cache.updateOwner(stream, newOwner);
- assertEquals("Owner of " + stream + " should be changed from " + oldOwner + " to " + newOwner,
- newOwner, cache.getOwner(stream));
- ownershipMap = cache.getStreamOwnerMapping();
- assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
- numProxies * numStreamsPerProxy, ownershipMap.size());
- assertEquals("Owner of " + stream + " should be " + newOwner,
- newOwner, ownershipMap.get(stream));
- ownershipDistribution = cache.getStreamOwnershipDistribution();
- assertEquals("There should be " + (numProxies + 1) + " proxies cached",
- numProxies + 1, ownershipDistribution.size());
- Set<String> oldOwnedStreams = ownershipDistribution.get(oldOwner);
- assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned by " + oldOwner,
- numStreamsPerProxy - 1, oldOwnedStreams.size());
- assertFalse("Stream " + stream + " should not be owned by " + oldOwner,
- oldOwnedStreams.contains(stream));
- Set<String> newOwnedStreams = ownershipDistribution.get(newOwner);
- assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned by " + newOwner,
- 1, newOwnedStreams.size());
- assertTrue("Stream " + stream + " should be owned by " + newOwner,
- newOwnedStreams.contains(stream));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockDistributedLogServices.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockDistributedLogServices.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockDistributedLogServices.java
deleted file mode 100644
index f088c0d..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockDistributedLogServices.java
+++ /dev/null
@@ -1,144 +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.client.proxy;
-
-import com.twitter.distributedlog.thrift.service.BulkWriteResponse;
-import com.twitter.distributedlog.thrift.service.ClientInfo;
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.distributedlog.thrift.service.HeartbeatOptions;
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import com.twitter.distributedlog.thrift.service.WriteContext;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.util.Future;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-/**
- * Mock DistributedLog Related Services.
- */
-public class MockDistributedLogServices {
-
- /**
- * Mock basic service.
- */
- static class MockBasicService implements DistributedLogService.ServiceIface {
-
- @Override
- public Future<ServerInfo> handshake() {
- return Future.value(new ServerInfo());
- }
-
- @Override
- public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
- return Future.value(new ServerInfo());
- }
-
- @Override
- public Future<WriteResponse> heartbeat(String stream, WriteContext ctx) {
- return Future.value(new WriteResponse());
- }
-
- @Override
- public Future<WriteResponse> heartbeatWithOptions(String stream,
- WriteContext ctx,
- HeartbeatOptions options) {
- return Future.value(new WriteResponse());
- }
-
- @Override
- public Future<WriteResponse> write(String stream,
- ByteBuffer data) {
- return Future.value(new WriteResponse());
- }
-
- @Override
- public Future<WriteResponse> writeWithContext(String stream,
- ByteBuffer data,
- WriteContext ctx) {
- return Future.value(new WriteResponse());
- }
-
- @Override
- public Future<BulkWriteResponse> writeBulkWithContext(String stream,
- List<ByteBuffer> data,
- WriteContext ctx) {
- return Future.value(new BulkWriteResponse());
- }
-
- @Override
- public Future<WriteResponse> truncate(String stream,
- String dlsn,
- WriteContext ctx) {
- return Future.value(new WriteResponse());
- }
-
- @Override
- public Future<WriteResponse> release(String stream,
- WriteContext ctx) {
- return Future.value(new WriteResponse());
- }
-
- @Override
- public Future<WriteResponse> create(String stream, WriteContext ctx) {
- return Future.value(new WriteResponse());
- }
-
- @Override
- public Future<WriteResponse> delete(String stream,
- WriteContext ctx) {
- return Future.value(new WriteResponse());
- }
-
- @Override
- public Future<WriteResponse> getOwner(String stream, WriteContext ctx) {
- return Future.value(new WriteResponse());
- }
-
- @Override
- public Future<Void> setAcceptNewStream(boolean enabled) {
- return Future.value(null);
- }
- }
-
- /**
- * Mock server info service.
- */
- public static class MockServerInfoService extends MockBasicService {
-
- protected ServerInfo serverInfo;
-
- public MockServerInfoService() {
- serverInfo = new ServerInfo();
- }
-
- public void updateServerInfo(ServerInfo serverInfo) {
- this.serverInfo = serverInfo;
- }
-
- @Override
- public Future<ServerInfo> handshake() {
- return Future.value(serverInfo);
- }
-
- @Override
- public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
- return Future.value(serverInfo);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockProxyClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockProxyClientBuilder.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockProxyClientBuilder.java
deleted file mode 100644
index ff0bd05..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockProxyClientBuilder.java
+++ /dev/null
@@ -1,49 +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.client.proxy;
-
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import java.net.SocketAddress;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Mock Proxy Client Builder.
- */
-class MockProxyClientBuilder implements ProxyClient.Builder {
-
- static class MockProxyClient extends ProxyClient {
- MockProxyClient(SocketAddress address,
- DistributedLogService.ServiceIface service) {
- super(address, new MockThriftClient(), service);
- }
- }
-
- private final ConcurrentMap<SocketAddress, MockProxyClient> clients =
- new ConcurrentHashMap<SocketAddress, MockProxyClient>();
-
- public void provideProxyClient(SocketAddress address,
- MockProxyClient proxyClient) {
- clients.put(address, proxyClient);
- }
-
- @Override
- public ProxyClient build(SocketAddress address) {
- return clients.get(address);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockThriftClient.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockThriftClient.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockThriftClient.java
deleted file mode 100644
index 7877ed7..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockThriftClient.java
+++ /dev/null
@@ -1,32 +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.client.proxy;
-
-import com.twitter.finagle.Service;
-import com.twitter.finagle.thrift.ThriftClientRequest;
-import com.twitter.util.Future;
-
-/**
- * Mock Thrift Client.
- */
-class MockThriftClient extends Service<ThriftClientRequest, byte[]> {
- @Override
- public Future<byte[]> apply(ThriftClientRequest request) {
- return Future.value(request.message);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/TestProxyClientManager.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/TestProxyClientManager.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/TestProxyClientManager.java
deleted file mode 100644
index 11e1e58..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/TestProxyClientManager.java
+++ /dev/null
@@ -1,368 +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.client.proxy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.proxy.MockDistributedLogServices.MockBasicService;
-import com.twitter.distributedlog.client.proxy.MockDistributedLogServices.MockServerInfoService;
-import com.twitter.distributedlog.client.proxy.MockProxyClientBuilder.MockProxyClient;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.client.stats.ClientStats;
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.commons.lang3.tuple.Pair;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test Proxy Client Manager.
- */
-public class TestProxyClientManager {
-
- @Rule
- public TestName runtime = new TestName();
-
- static class TestHostProvider implements HostProvider {
-
- Set<SocketAddress> hosts = new HashSet<SocketAddress>();
-
- synchronized void addHost(SocketAddress host) {
- hosts.add(host);
- }
-
- @Override
- public synchronized Set<SocketAddress> getHosts() {
- return ImmutableSet.copyOf(hosts);
- }
-
- }
-
- private static ProxyClientManager createProxyClientManager(ProxyClient.Builder builder,
- long periodicHandshakeIntervalMs) {
- HostProvider provider = new TestHostProvider();
- return createProxyClientManager(builder, provider, periodicHandshakeIntervalMs);
- }
-
- private static ProxyClientManager createProxyClientManager(ProxyClient.Builder builder,
- HostProvider hostProvider,
- long periodicHandshakeIntervalMs) {
- ClientConfig clientConfig = new ClientConfig();
- clientConfig.setPeriodicHandshakeIntervalMs(periodicHandshakeIntervalMs);
- clientConfig.setPeriodicOwnershipSyncIntervalMs(-1);
- HashedWheelTimer dlTimer = new HashedWheelTimer(
- new ThreadFactoryBuilder().setNameFormat("TestProxyClientManager-timer-%d").build(),
- clientConfig.getRedirectBackoffStartMs(),
- TimeUnit.MILLISECONDS);
- return new ProxyClientManager(clientConfig, builder, dlTimer, hostProvider,
- new ClientStats(NullStatsReceiver.get(), false, new DefaultRegionResolver()));
- }
-
- private static SocketAddress createSocketAddress(int port) {
- return new InetSocketAddress("127.0.0.1", port);
- }
-
- private static MockProxyClient createMockProxyClient(SocketAddress address) {
- return new MockProxyClient(address, new MockBasicService());
- }
-
- private static Pair<MockProxyClient, MockServerInfoService> createMockProxyClient(
- SocketAddress address, ServerInfo serverInfo) {
- MockServerInfoService service = new MockServerInfoService();
- MockProxyClient proxyClient = new MockProxyClient(address, service);
- service.updateServerInfo(serverInfo);
- return Pair.of(proxyClient, service);
- }
-
- @Test(timeout = 60000)
- public void testBasicCreateRemove() throws Exception {
- SocketAddress address = createSocketAddress(1000);
- MockProxyClientBuilder builder = new MockProxyClientBuilder();
- MockProxyClient mockProxyClient = createMockProxyClient(address);
- builder.provideProxyClient(address, mockProxyClient);
-
- ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
- assertEquals("There should be no clients in the manager",
- 0, clientManager.getNumProxies());
- ProxyClient proxyClient = clientManager.createClient(address);
- assertEquals("Create client should build the proxy client",
- 1, clientManager.getNumProxies());
- assertTrue("The client returned should be the same client that builder built",
- mockProxyClient == proxyClient);
- }
-
- @Test(timeout = 60000)
- public void testGetShouldCreateClient() throws Exception {
- SocketAddress address = createSocketAddress(2000);
- MockProxyClientBuilder builder = new MockProxyClientBuilder();
- MockProxyClient mockProxyClient = createMockProxyClient(address);
- builder.provideProxyClient(address, mockProxyClient);
-
- ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
- assertEquals("There should be no clients in the manager",
- 0, clientManager.getNumProxies());
- ProxyClient proxyClient = clientManager.getClient(address);
- assertEquals("Get client should build the proxy client",
- 1, clientManager.getNumProxies());
- assertTrue("The client returned should be the same client that builder built",
- mockProxyClient == proxyClient);
- }
-
- @Test(timeout = 60000)
- public void testConditionalRemoveClient() throws Exception {
- SocketAddress address = createSocketAddress(3000);
- MockProxyClientBuilder builder = new MockProxyClientBuilder();
- MockProxyClient mockProxyClient = createMockProxyClient(address);
- MockProxyClient anotherMockProxyClient = createMockProxyClient(address);
- builder.provideProxyClient(address, mockProxyClient);
-
- ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
- assertEquals("There should be no clients in the manager",
- 0, clientManager.getNumProxies());
- clientManager.createClient(address);
- assertEquals("Create client should build the proxy client",
- 1, clientManager.getNumProxies());
- clientManager.removeClient(address, anotherMockProxyClient);
- assertEquals("Conditional remove should not remove proxy client",
- 1, clientManager.getNumProxies());
- clientManager.removeClient(address, mockProxyClient);
- assertEquals("Conditional remove should remove proxy client",
- 0, clientManager.getNumProxies());
- }
-
- @Test(timeout = 60000)
- public void testRemoveClient() throws Exception {
- SocketAddress address = createSocketAddress(3000);
- MockProxyClientBuilder builder = new MockProxyClientBuilder();
- MockProxyClient mockProxyClient = createMockProxyClient(address);
- builder.provideProxyClient(address, mockProxyClient);
-
- ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
- assertEquals("There should be no clients in the manager",
- 0, clientManager.getNumProxies());
- clientManager.createClient(address);
- assertEquals("Create client should build the proxy client",
- 1, clientManager.getNumProxies());
- clientManager.removeClient(address);
- assertEquals("Remove should remove proxy client",
- 0, clientManager.getNumProxies());
- }
-
- @Test(timeout = 60000)
- public void testCreateClientShouldHandshake() throws Exception {
- SocketAddress address = createSocketAddress(3000);
- MockProxyClientBuilder builder = new MockProxyClientBuilder();
- ServerInfo serverInfo = new ServerInfo();
- serverInfo.putToOwnerships(runtime.getMethodName() + "_stream",
- runtime.getMethodName() + "_owner");
- Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
- createMockProxyClient(address, serverInfo);
- builder.provideProxyClient(address, mockProxyClient.getLeft());
-
- final AtomicReference<ServerInfo> resultHolder = new AtomicReference<ServerInfo>(null);
- final CountDownLatch doneLatch = new CountDownLatch(1);
- ProxyListener listener = new ProxyListener() {
- @Override
- public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
- resultHolder.set(serverInfo);
- doneLatch.countDown();
- }
- @Override
- public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
- }
- };
-
- ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
- clientManager.registerProxyListener(listener);
- assertEquals("There should be no clients in the manager",
- 0, clientManager.getNumProxies());
- clientManager.createClient(address);
- assertEquals("Create client should build the proxy client",
- 1, clientManager.getNumProxies());
-
- // When a client is created, it would handshake with that proxy
- doneLatch.await();
- assertEquals("Handshake should return server info",
- serverInfo, resultHolder.get());
- }
-
- @Test(timeout = 60000)
- public void testHandshake() throws Exception {
- final int numHosts = 3;
- final int numStreamsPerHost = 3;
- final int initialPort = 4000;
-
- MockProxyClientBuilder builder = new MockProxyClientBuilder();
- Map<SocketAddress, ServerInfo> serverInfoMap =
- new HashMap<SocketAddress, ServerInfo>();
- for (int i = 0; i < numHosts; i++) {
- SocketAddress address = createSocketAddress(initialPort + i);
- ServerInfo serverInfo = new ServerInfo();
- for (int j = 0; j < numStreamsPerHost; j++) {
- serverInfo.putToOwnerships(runtime.getMethodName() + "_stream_" + j,
- address.toString());
- }
- Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
- createMockProxyClient(address, serverInfo);
- builder.provideProxyClient(address, mockProxyClient.getLeft());
- serverInfoMap.put(address, serverInfo);
- }
-
- final Map<SocketAddress, ServerInfo> results = new HashMap<SocketAddress, ServerInfo>();
- final CountDownLatch doneLatch = new CountDownLatch(2 * numHosts);
- ProxyListener listener = new ProxyListener() {
- @Override
- public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
- synchronized (results) {
- results.put(address, serverInfo);
- }
- doneLatch.countDown();
- }
-
- @Override
- public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
- }
- };
-
- TestHostProvider rs = new TestHostProvider();
- ProxyClientManager clientManager = createProxyClientManager(builder, rs, 0L);
- clientManager.registerProxyListener(listener);
- assertEquals("There should be no clients in the manager",
- 0, clientManager.getNumProxies());
- for (int i = 0; i < numHosts; i++) {
- rs.addHost(createSocketAddress(initialPort + i));
- }
- // handshake would handshake with 3 hosts again
- clientManager.handshake();
- doneLatch.await();
- assertEquals("Handshake should return server info",
- numHosts, results.size());
- assertTrue("Handshake should get all server infos",
- Maps.difference(serverInfoMap, results).areEqual());
- }
-
- @Test(timeout = 60000)
- public void testPeriodicHandshake() throws Exception {
- final int numHosts = 3;
- final int numStreamsPerHost = 3;
- final int initialPort = 5000;
-
- MockProxyClientBuilder builder = new MockProxyClientBuilder();
- Map<SocketAddress, ServerInfo> serverInfoMap =
- new HashMap<SocketAddress, ServerInfo>();
- Map<SocketAddress, MockServerInfoService> mockServiceMap =
- new HashMap<SocketAddress, MockServerInfoService>();
- final Map<SocketAddress, CountDownLatch> hostDoneLatches =
- new HashMap<SocketAddress, CountDownLatch>();
- for (int i = 0; i < numHosts; i++) {
- SocketAddress address = createSocketAddress(initialPort + i);
- ServerInfo serverInfo = new ServerInfo();
- for (int j = 0; j < numStreamsPerHost; j++) {
- serverInfo.putToOwnerships(runtime.getMethodName() + "_stream_" + j,
- address.toString());
- }
- Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
- createMockProxyClient(address, serverInfo);
- builder.provideProxyClient(address, mockProxyClient.getLeft());
- serverInfoMap.put(address, serverInfo);
- mockServiceMap.put(address, mockProxyClient.getRight());
- hostDoneLatches.put(address, new CountDownLatch(2));
- }
-
- final Map<SocketAddress, ServerInfo> results = new HashMap<SocketAddress, ServerInfo>();
- final CountDownLatch doneLatch = new CountDownLatch(numHosts);
- ProxyListener listener = new ProxyListener() {
- @Override
- public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
- synchronized (results) {
- results.put(address, serverInfo);
- CountDownLatch latch = hostDoneLatches.get(address);
- if (null != latch) {
- latch.countDown();
- }
- }
- doneLatch.countDown();
- }
-
- @Override
- public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
- }
- };
-
- TestHostProvider rs = new TestHostProvider();
- ProxyClientManager clientManager = createProxyClientManager(builder, rs, 50L);
- clientManager.setPeriodicHandshakeEnabled(false);
- clientManager.registerProxyListener(listener);
-
- assertEquals("There should be no clients in the manager",
- 0, clientManager.getNumProxies());
- for (int i = 0; i < numHosts; i++) {
- SocketAddress address = createSocketAddress(initialPort + i);
- rs.addHost(address);
- clientManager.createClient(address);
- }
-
- // make sure the first 3 handshakes going through
- doneLatch.await();
-
- assertEquals("Handshake should return server info",
- numHosts, results.size());
- assertTrue("Handshake should get all server infos",
- Maps.difference(serverInfoMap, results).areEqual());
-
- // update server info
- for (int i = 0; i < numHosts; i++) {
- SocketAddress address = createSocketAddress(initialPort + i);
- ServerInfo serverInfo = new ServerInfo();
- for (int j = 0; j < numStreamsPerHost; j++) {
- serverInfo.putToOwnerships(runtime.getMethodName() + "_new_stream_" + j,
- address.toString());
- }
- MockServerInfoService service = mockServiceMap.get(address);
- serverInfoMap.put(address, serverInfo);
- service.updateServerInfo(serverInfo);
- }
-
- clientManager.setPeriodicHandshakeEnabled(true);
- for (int i = 0; i < numHosts; i++) {
- SocketAddress address = createSocketAddress(initialPort + i);
- CountDownLatch latch = hostDoneLatches.get(address);
- latch.await();
- }
-
- assertTrue("Periodic handshake should update all server infos",
- Maps.difference(serverInfoMap, results).areEqual());
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestConsistentHashRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestConsistentHashRoutingService.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestConsistentHashRoutingService.java
deleted file mode 100644
index 0f4804c..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestConsistentHashRoutingService.java
+++ /dev/null
@@ -1,417 +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.client.routing;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.finagle.Address;
-import com.twitter.finagle.Addresses;
-import com.twitter.finagle.ChannelWriteException;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ConsistentHashRoutingService}.
- */
-public class TestConsistentHashRoutingService {
-
- @Test(timeout = 60000)
- public void testBlackoutHost() throws Exception {
- TestName name = new TestName();
- RoutingService routingService = ConsistentHashRoutingService.newBuilder()
- .serverSet(new NameServerSet(name))
- .resolveFromName(true)
- .numReplicas(997)
- .blackoutSeconds(2)
- .build();
-
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", 3181);
- Address address = Addresses.newInetAddress(inetAddress);
- List<Address> addresses = new ArrayList<Address>(1);
- addresses.add(address);
- name.changeAddrs(addresses);
-
- routingService.startService();
-
- RoutingService.RoutingContext routingContext =
- RoutingService.RoutingContext.of(new DefaultRegionResolver());
-
- String streamName = "test-blackout-host";
- assertEquals(inetAddress, routingService.getHost(streamName, routingContext));
- routingService.removeHost(inetAddress, new ChannelWriteException(new IOException("test exception")));
- try {
- routingService.getHost(streamName, routingContext);
- fail("Should fail to get host since no brokers are available");
- } catch (NoBrokersAvailableException nbae) {
- // expected
- }
-
- TimeUnit.SECONDS.sleep(3);
- assertEquals(inetAddress, routingService.getHost(streamName, routingContext));
-
- routingService.stopService();
- }
-
- @Test(timeout = 60000)
- public void testPerformServerSetChangeOnName() throws Exception {
- TestName name = new TestName();
- ConsistentHashRoutingService routingService = (ConsistentHashRoutingService)
- ConsistentHashRoutingService.newBuilder()
- .serverSet(new NameServerSet(name))
- .resolveFromName(true)
- .numReplicas(997)
- .build();
-
- int basePort = 3180;
- int numHosts = 4;
- List<Address> addresses1 = Lists.newArrayListWithExpectedSize(4);
- List<Address> addresses2 = Lists.newArrayListWithExpectedSize(4);
- List<Address> addresses3 = Lists.newArrayListWithExpectedSize(4);
-
- // fill up the addresses1
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
- Address address = Addresses.newInetAddress(inetAddress);
- addresses1.add(address);
- }
- // fill up the addresses2 - overlap with addresses1
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 2 + i);
- Address address = Addresses.newInetAddress(inetAddress);
- addresses2.add(address);
- }
- // fill up the addresses3 - not overlap with addresses2
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
- Address address = Addresses.newInetAddress(inetAddress);
- addresses3.add(address);
- }
-
- final List<SocketAddress> leftAddresses = Lists.newArrayList();
- final List<SocketAddress> joinAddresses = Lists.newArrayList();
-
- RoutingService.RoutingListener routingListener = new RoutingService.RoutingListener() {
- @Override
- public void onServerLeft(SocketAddress address) {
- synchronized (leftAddresses) {
- leftAddresses.add(address);
- leftAddresses.notifyAll();
- }
- }
-
- @Override
- public void onServerJoin(SocketAddress address) {
- synchronized (joinAddresses) {
- joinAddresses.add(address);
- joinAddresses.notifyAll();
- }
- }
- };
-
- routingService.registerListener(routingListener);
- name.changeAddrs(addresses1);
-
- routingService.startService();
-
- synchronized (joinAddresses) {
- while (joinAddresses.size() < numHosts) {
- joinAddresses.wait();
- }
- }
-
- // validate 4 nodes joined
- synchronized (joinAddresses) {
- assertEquals(numHosts, joinAddresses.size());
- }
- synchronized (leftAddresses) {
- assertEquals(0, leftAddresses.size());
- }
- assertEquals(numHosts, routingService.shardId2Address.size());
- assertEquals(numHosts, routingService.address2ShardId.size());
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
- assertTrue(routingService.address2ShardId.containsKey(inetAddress));
- int shardId = routingService.address2ShardId.get(inetAddress);
- SocketAddress sa = routingService.shardId2Address.get(shardId);
- assertNotNull(sa);
- assertEquals(inetAddress, sa);
- }
-
- // update addresses2 - 2 new hosts joined, 2 old hosts left
- name.changeAddrs(addresses2);
- synchronized (joinAddresses) {
- while (joinAddresses.size() < numHosts + 2) {
- joinAddresses.wait();
- }
- }
- synchronized (leftAddresses) {
- while (leftAddresses.size() < numHosts - 2) {
- leftAddresses.wait();
- }
- }
- assertEquals(numHosts, routingService.shardId2Address.size());
- assertEquals(numHosts, routingService.address2ShardId.size());
-
- // first 2 shards should leave
- for (int i = 0; i < 2; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
- assertFalse(routingService.address2ShardId.containsKey(inetAddress));
- }
-
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 2 + i);
- assertTrue(routingService.address2ShardId.containsKey(inetAddress));
- int shardId = routingService.address2ShardId.get(inetAddress);
- SocketAddress sa = routingService.shardId2Address.get(shardId);
- assertNotNull(sa);
- assertEquals(inetAddress, sa);
- }
-
- // update addresses3 - 2 new hosts joined, 2 old hosts left
- name.changeAddrs(addresses3);
- synchronized (joinAddresses) {
- while (joinAddresses.size() < numHosts + 2 + numHosts) {
- joinAddresses.wait();
- }
- }
- synchronized (leftAddresses) {
- while (leftAddresses.size() < numHosts - 2 + numHosts) {
- leftAddresses.wait();
- }
- }
- assertEquals(numHosts, routingService.shardId2Address.size());
- assertEquals(numHosts, routingService.address2ShardId.size());
-
- // first 6 shards should leave
- for (int i = 0; i < 2 + numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
- assertFalse(routingService.address2ShardId.containsKey(inetAddress));
- }
- // new 4 shards should exist
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
- assertTrue(routingService.address2ShardId.containsKey(inetAddress));
- int shardId = routingService.address2ShardId.get(inetAddress);
- SocketAddress sa = routingService.shardId2Address.get(shardId);
- assertNotNull(sa);
- assertEquals(inetAddress, sa);
- }
-
- }
-
- private static class TestServerSetWatcher implements ServerSetWatcher {
-
- final LinkedBlockingQueue<ImmutableSet<DLSocketAddress>> changeQueue =
- new LinkedBlockingQueue<ImmutableSet<DLSocketAddress>>();
- final CopyOnWriteArrayList<ServerSetMonitor> monitors =
- new CopyOnWriteArrayList<ServerSetMonitor>();
-
- @Override
- public void watch(ServerSetMonitor monitor) throws MonitorException {
- monitors.add(monitor);
- ImmutableSet<DLSocketAddress> change;
- while ((change = changeQueue.poll()) != null) {
- notifyChanges(change);
- }
- }
-
- void notifyChanges(ImmutableSet<DLSocketAddress> addresses) {
- if (monitors.isEmpty()) {
- changeQueue.add(addresses);
- } else {
- for (ServerSetMonitor monitor : monitors) {
- monitor.onChange(addresses);
- }
- }
- }
- }
-
- @Test(timeout = 60000)
- public void testPerformServerSetChangeOnServerSet() throws Exception {
- TestServerSetWatcher serverSetWatcher = new TestServerSetWatcher();
- ConsistentHashRoutingService routingService = new ConsistentHashRoutingService(
- serverSetWatcher, 997, Integer.MAX_VALUE, NullStatsReceiver.get());
-
- int basePort = 3180;
- int numHosts = 4;
- Set<DLSocketAddress> addresses1 = Sets.newConcurrentHashSet();
- Set<DLSocketAddress> addresses2 = Sets.newConcurrentHashSet();
- Set<DLSocketAddress> addresses3 = Sets.newConcurrentHashSet();
-
- // fill up the addresses1
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
- DLSocketAddress dsa = new DLSocketAddress(i, inetAddress);
- addresses1.add(dsa);
- }
- // fill up the addresses2 - overlap with addresses1
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + i);
- DLSocketAddress dsa = new DLSocketAddress(i + 2, inetAddress);
- addresses2.add(dsa);
- }
- // fill up the addresses3 - not overlap with addresses2
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
- DLSocketAddress dsa = new DLSocketAddress(i, inetAddress);
- addresses3.add(dsa);
- }
-
- final List<SocketAddress> leftAddresses = Lists.newArrayList();
- final List<SocketAddress> joinAddresses = Lists.newArrayList();
-
- RoutingService.RoutingListener routingListener = new RoutingService.RoutingListener() {
- @Override
- public void onServerLeft(SocketAddress address) {
- synchronized (leftAddresses) {
- leftAddresses.add(address);
- leftAddresses.notifyAll();
- }
- }
-
- @Override
- public void onServerJoin(SocketAddress address) {
- synchronized (joinAddresses) {
- joinAddresses.add(address);
- joinAddresses.notifyAll();
- }
- }
- };
-
- routingService.registerListener(routingListener);
- serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses1));
-
- routingService.startService();
-
- synchronized (joinAddresses) {
- while (joinAddresses.size() < numHosts) {
- joinAddresses.wait();
- }
- }
-
- // validate 4 nodes joined
- synchronized (joinAddresses) {
- assertEquals(numHosts, joinAddresses.size());
- }
- synchronized (leftAddresses) {
- assertEquals(0, leftAddresses.size());
- }
- assertEquals(numHosts, routingService.shardId2Address.size());
- assertEquals(numHosts, routingService.address2ShardId.size());
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
- assertTrue(routingService.address2ShardId.containsKey(inetAddress));
- int shardId = routingService.address2ShardId.get(inetAddress);
- assertEquals(i, shardId);
- SocketAddress sa = routingService.shardId2Address.get(shardId);
- assertNotNull(sa);
- assertEquals(inetAddress, sa);
- }
-
- // update addresses2 - 2 new hosts joined, 2 old hosts left
- serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses2));
- synchronized (joinAddresses) {
- while (joinAddresses.size() < numHosts + 2) {
- joinAddresses.wait();
- }
- }
- synchronized (leftAddresses) {
- while (leftAddresses.size() < 2) {
- leftAddresses.wait();
- }
- }
-
- assertEquals(numHosts + 2, routingService.shardId2Address.size());
- assertEquals(numHosts + 2, routingService.address2ShardId.size());
- // first 2 shards should not leave
- for (int i = 0; i < 2; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
- assertTrue(routingService.address2ShardId.containsKey(inetAddress));
- int shardId = routingService.address2ShardId.get(inetAddress);
- assertEquals(i, shardId);
- SocketAddress sa = routingService.shardId2Address.get(shardId);
- assertNotNull(sa);
- assertEquals(inetAddress, sa);
- }
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + i);
- assertTrue(routingService.address2ShardId.containsKey(inetAddress));
- int shardId = routingService.address2ShardId.get(inetAddress);
- assertEquals(i + 2, shardId);
- SocketAddress sa = routingService.shardId2Address.get(shardId);
- assertNotNull(sa);
- assertEquals(inetAddress, sa);
- }
-
- // update addresses3
- serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses3));
- synchronized (joinAddresses) {
- while (joinAddresses.size() < numHosts + 2 + numHosts) {
- joinAddresses.wait();
- }
- }
- synchronized (leftAddresses) {
- while (leftAddresses.size() < 2 + numHosts) {
- leftAddresses.wait();
- }
- }
- assertEquals(numHosts + 2, routingService.shardId2Address.size());
- assertEquals(numHosts + 2, routingService.address2ShardId.size());
-
- // first 4 shards should leave
- for (int i = 0; i < numHosts; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
- assertTrue(routingService.address2ShardId.containsKey(inetAddress));
- int shardId = routingService.address2ShardId.get(inetAddress);
- assertEquals(i, shardId);
- SocketAddress sa = routingService.shardId2Address.get(shardId);
- assertNotNull(sa);
- assertEquals(inetAddress, sa);
- }
- // the other 2 shards should be still there
- for (int i = 0; i < 2; i++) {
- InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + 2 + i);
- assertTrue(routingService.address2ShardId.containsKey(inetAddress));
- int shardId = routingService.address2ShardId.get(inetAddress);
- assertEquals(numHosts + i, shardId);
- SocketAddress sa = routingService.shardId2Address.get(shardId);
- assertNotNull(sa);
- assertEquals(inetAddress, sa);
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestInetNameResolution.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestInetNameResolution.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestInetNameResolution.java
deleted file mode 100644
index 2552f9e..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestInetNameResolution.java
+++ /dev/null
@@ -1,73 +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.client.routing;
-
-import com.google.common.collect.ImmutableSet;
-import com.twitter.common.net.pool.DynamicHostSet;
-import com.twitter.thrift.Endpoint;
-import com.twitter.thrift.ServiceInstance;
-import java.net.InetSocketAddress;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for `inet` name resolution.
- */
-public class TestInetNameResolution {
-
- private static final Logger logger = LoggerFactory.getLogger(TestRoutingService.class);
-
- @Test(timeout = 10000)
- public void testInetNameResolution() throws Exception {
- String nameStr = "inet!127.0.0.1:3181";
- final CountDownLatch resolved = new CountDownLatch(1);
- final AtomicBoolean validationFailed = new AtomicBoolean(false);
-
- NameServerSet serverSet = new NameServerSet(nameStr);
- serverSet.watch(new DynamicHostSet.HostChangeMonitor<ServiceInstance>() {
- @Override
- public void onChange(ImmutableSet<ServiceInstance> hostSet) {
- if (hostSet.size() > 1) {
- logger.error("HostSet has more elements than expected {}", hostSet);
- validationFailed.set(true);
- resolved.countDown();
- } else if (hostSet.size() == 1) {
- ServiceInstance serviceInstance = hostSet.iterator().next();
- Endpoint endpoint = serviceInstance.getAdditionalEndpoints().get("thrift");
- InetSocketAddress address = new InetSocketAddress(endpoint.getHost(), endpoint.getPort());
- if (endpoint.getPort() != 3181) {
- logger.error("Port does not match the expected port {}", endpoint.getPort());
- validationFailed.set(true);
- } else if (!address.getAddress().getHostAddress().equals("127.0.0.1")) {
- logger.error("Host address does not match the expected address {}",
- address.getAddress().getHostAddress());
- validationFailed.set(true);
- }
- resolved.countDown();
- }
- }
- });
-
- resolved.await();
- Assert.assertEquals(false, validationFailed.get());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRegionsRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRegionsRoutingService.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRegionsRoutingService.java
deleted file mode 100644
index 49a375c..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRegionsRoutingService.java
+++ /dev/null
@@ -1,133 +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.client.routing;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.finagle.NoBrokersAvailableException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.junit.Test;
-
-/**
- * Test Case for {@link RegionsRoutingService}.
- */
-public class TestRegionsRoutingService {
-
- @Test(timeout = 60000)
- public void testRoutingListener() throws Exception {
- int numRoutingServices = 5;
- RoutingService.Builder[] routingServiceBuilders = new RoutingService.Builder[numRoutingServices];
- Set<SocketAddress> hosts = new HashSet<SocketAddress>();
- Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
- for (int i = 0; i < numRoutingServices; i++) {
- String finagleNameStr = "inet!127.0.0.1:" + (3181 + i);
- routingServiceBuilders[i] = RoutingUtils.buildRoutingService(finagleNameStr);
- SocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
- hosts.add(address);
- regionMap.put(address, "region-" + i);
- }
-
- final CountDownLatch doneLatch = new CountDownLatch(numRoutingServices);
- final AtomicInteger numHostsLeft = new AtomicInteger(0);
- final Set<SocketAddress> jointHosts = new HashSet<SocketAddress>();
- RegionsRoutingService regionsRoutingService =
- RegionsRoutingService.newBuilder()
- .routingServiceBuilders(routingServiceBuilders)
- .resolver(new DefaultRegionResolver(regionMap))
- .build();
- regionsRoutingService.registerListener(new RoutingService.RoutingListener() {
- @Override
- public void onServerLeft(SocketAddress address) {
- numHostsLeft.incrementAndGet();
- }
-
- @Override
- public void onServerJoin(SocketAddress address) {
- jointHosts.add(address);
- doneLatch.countDown();
- }
- });
-
- regionsRoutingService.startService();
-
- doneLatch.await();
-
- assertEquals(numRoutingServices, jointHosts.size());
- assertEquals(0, numHostsLeft.get());
- assertTrue(Sets.difference(hosts, jointHosts).immutableCopy().isEmpty());
- }
-
- @Test(timeout = 60000)
- public void testGetHost() throws Exception {
- int numRoutingServices = 3;
- RoutingService.Builder[] routingServiceBuilders = new RoutingService.Builder[numRoutingServices];
- Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
- for (int i = 0; i < numRoutingServices; i++) {
- String finagleNameStr = "inet!127.0.0.1:" + (3181 + i);
- routingServiceBuilders[i] = RoutingUtils.buildRoutingService(finagleNameStr);
- SocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
- regionMap.put(address, "region-" + i);
- }
-
- RegionsRoutingService regionsRoutingService =
- RegionsRoutingService.newBuilder()
- .resolver(new DefaultRegionResolver(regionMap))
- .routingServiceBuilders(routingServiceBuilders)
- .build();
- regionsRoutingService.startService();
-
- RoutingService.RoutingContext routingContext =
- RoutingService.RoutingContext.of(new DefaultRegionResolver())
- .addTriedHost(new InetSocketAddress("127.0.0.1", 3183), StatusCode.WRITE_EXCEPTION);
- assertEquals(new InetSocketAddress("127.0.0.1", 3181),
- regionsRoutingService.getHost("any", routingContext));
-
- routingContext =
- RoutingService.RoutingContext.of(new DefaultRegionResolver())
- .addTriedHost(new InetSocketAddress("127.0.0.1", 3181), StatusCode.WRITE_EXCEPTION);
- assertEquals(new InetSocketAddress("127.0.0.1", 3182),
- regionsRoutingService.getHost("any", routingContext));
-
- // add 3182 to routing context as tried host
- routingContext.addTriedHost(new InetSocketAddress("127.0.0.1", 3182), StatusCode.WRITE_EXCEPTION);
- assertEquals(new InetSocketAddress("127.0.0.1", 3183),
- regionsRoutingService.getHost("any", routingContext));
-
- // add 3183 to routing context as tried host
- routingContext.addTriedHost(new InetSocketAddress("127.0.0.1", 3183), StatusCode.WRITE_EXCEPTION);
- try {
- regionsRoutingService.getHost("any", routingContext);
- fail("Should fail to get host since all regions are tried.");
- } catch (NoBrokersAvailableException nbae) {
- // expected
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRoutingService.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRoutingService.java
deleted file mode 100644
index b79557e..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRoutingService.java
+++ /dev/null
@@ -1,146 +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.client.routing;
-
-import static org.junit.Assert.assertEquals;
-
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.finagle.Address;
-import com.twitter.finagle.Addresses;
-import com.twitter.finagle.addr.WeightedAddress;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for {@link RoutingService}.
- */
-@RunWith(Parameterized.class)
-public class TestRoutingService {
-
- static final Logger LOG = LoggerFactory.getLogger(TestRoutingService.class);
-
- @Parameterized.Parameters
- public static Collection<Object[]> configs() {
- ArrayList<Object[]> list = new ArrayList<Object[]>();
- for (int i = 0; i <= 1; i++) {
- for (int j = 0; j <= 1; j++) {
- for (int k = 0; k <= 1; k++) {
- list.add(new Boolean[] {i == 1, j == 1, k == 1});
- }
- }
- }
- return list;
- }
-
- private final boolean consistentHash;
- private final boolean weightedAddresses;
- private final boolean asyncResolution;
-
- public TestRoutingService(boolean consistentHash, boolean weightedAddresses, boolean asyncResolution) {
- this.consistentHash = consistentHash;
- this.weightedAddresses = weightedAddresses;
- this.asyncResolution = asyncResolution;
- }
-
- private List<Address> getAddresses(boolean weightedAddresses) {
- ArrayList<Address> addresses = new ArrayList<Address>();
- addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.1", 3181)));
- addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.2", 3181)));
- addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.3", 3181)));
- addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.4", 3181)));
- addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.5", 3181)));
- addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.6", 3181)));
- addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.7", 3181)));
-
- if (weightedAddresses) {
- ArrayList<Address> wAddresses = new ArrayList<Address>();
- for (Address address: addresses) {
- wAddresses.add(WeightedAddress.apply(address, 1.0));
- }
- return wAddresses;
- } else {
- return addresses;
- }
- }
-
- private void testRoutingServiceHelper(boolean consistentHash,
- boolean weightedAddresses,
- boolean asyncResolution)
- throws Exception {
- ExecutorService executorService = null;
- final List<Address> addresses = getAddresses(weightedAddresses);
- final TestName name = new TestName();
- RoutingService routingService;
- if (consistentHash) {
- routingService = ConsistentHashRoutingService.newBuilder()
- .serverSet(new NameServerSet(name))
- .resolveFromName(true)
- .numReplicas(997)
- .build();
- } else {
- routingService = ServerSetRoutingService.newServerSetRoutingServiceBuilder()
- .serverSetWatcher(new TwitterServerSetWatcher(new NameServerSet(name), true)).build();
- }
-
- if (asyncResolution) {
- executorService = Executors.newSingleThreadExecutor();
- executorService.submit(new Runnable() {
- @Override
- public void run() {
- name.changeAddrs(addresses);
- }
- });
- } else {
- name.changeAddrs(addresses);
- }
- routingService.startService();
-
- HashSet<SocketAddress> mapping = new HashSet<SocketAddress>();
-
- for (int i = 0; i < 1000; i++) {
- for (int j = 0; j < 5; j++) {
- String stream = "TestStream-" + i + "-" + j;
- mapping.add(routingService.getHost(stream,
- RoutingService.RoutingContext.of(new DefaultRegionResolver())));
- }
- }
-
- assertEquals(mapping.size(), addresses.size());
-
- if (null != executorService) {
- executorService.shutdown();
- }
-
- }
-
- @Test(timeout = 5000)
- public void testRoutingService() throws Exception {
- testRoutingServiceHelper(this.consistentHash, this.weightedAddresses, this.asyncResolution);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
deleted file mode 100644
index 71d0b01..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
+++ /dev/null
@@ -1,105 +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.client.speculative;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-
-import com.twitter.util.CountDownLatch;
-import com.twitter.util.Future;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test {@link TestDefaultSpeculativeRequestExecutionPolicy}.
- */
-public class TestDefaultSpeculativeRequestExecutionPolicy {
-
- @Test(timeout = 20000, expected = IllegalArgumentException.class)
- public void testInvalidBackoffMultiplier() throws Exception {
- new DefaultSpeculativeRequestExecutionPolicy(100, 200, -1);
- }
-
- @Test(timeout = 20000, expected = IllegalArgumentException.class)
- public void testInvalidMaxSpeculativeTimeout() throws Exception {
- new DefaultSpeculativeRequestExecutionPolicy(100, Integer.MAX_VALUE, 2);
- }
-
- @Test(timeout = 20000)
- public void testSpeculativeRequests() throws Exception {
- DefaultSpeculativeRequestExecutionPolicy policy =
- new DefaultSpeculativeRequestExecutionPolicy(10, 10000, 2);
- SpeculativeRequestExecutor executor = mock(SpeculativeRequestExecutor.class);
-
- final AtomicInteger callCount = new AtomicInteger(0);
- final CountDownLatch latch = new CountDownLatch(3);
-
- Mockito.doAnswer(new Answer() {
- @Override
- public Object answer(InvocationOnMock invocation) throws Throwable {
- try {
- return Future.value(callCount.incrementAndGet() < 3);
- } finally {
- latch.countDown();
- }
- }
- }).when(executor).issueSpeculativeRequest();
-
- ScheduledExecutorService executorService =
- Executors.newSingleThreadScheduledExecutor();
- policy.initiateSpeculativeRequest(executorService, executor);
-
- latch.await();
-
- assertEquals(40, policy.getNextSpeculativeRequestTimeout());
- }
-
- @Test(timeout = 20000)
- public void testSpeculativeRequestsWithMaxTimeout() throws Exception {
- DefaultSpeculativeRequestExecutionPolicy policy =
- new DefaultSpeculativeRequestExecutionPolicy(10, 15, 2);
- SpeculativeRequestExecutor executor = mock(SpeculativeRequestExecutor.class);
-
- final AtomicInteger callCount = new AtomicInteger(0);
- final CountDownLatch latch = new CountDownLatch(3);
-
- Mockito.doAnswer(new Answer() {
- @Override
- public Object answer(InvocationOnMock invocation) throws Throwable {
- try {
- return Future.value(callCount.incrementAndGet() < 3);
- } finally {
- latch.countDown();
- }
- }
- }).when(executor).issueSpeculativeRequest();
-
- ScheduledExecutorService executorService =
- Executors.newSingleThreadScheduledExecutor();
- policy.initiateSpeculativeRequest(executorService, executor);
-
- latch.await();
-
- assertEquals(15, policy.getNextSpeculativeRequestTimeout());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/service/TestDistributedLogClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/service/TestDistributedLogClientBuilder.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/service/TestDistributedLogClientBuilder.java
deleted file mode 100644
index 986cdd3..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/service/TestDistributedLogClientBuilder.java
+++ /dev/null
@@ -1,49 +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.service;
-
-import static org.junit.Assert.assertFalse;
-
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration;
-import org.junit.Test;
-
-/**
- * Test Case of {@link com.twitter.distributedlog.service.DistributedLogClientBuilder}.
- */
-public class TestDistributedLogClientBuilder {
-
- @Test(timeout = 60000)
- public void testBuildClientsFromSameBuilder() throws Exception {
- DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
- .name("build-clients-from-same-builder")
- .clientId(ClientId$.MODULE$.apply("test-builder"))
- .finagleNameStr("inet!127.0.0.1:7001")
- .streamNameRegex(".*")
- .handshakeWithClientInfo(true)
- .clientBuilder(ClientBuilder.get()
- .hostConnectionLimit(1)
- .connectTimeout(Duration.fromSeconds(1))
- .tcpConnectTimeout(Duration.fromSeconds(1))
- .requestTimeout(Duration.fromSeconds(10)));
- DistributedLogClient client1 = builder.build();
- DistributedLogClient client2 = builder.build();
- assertFalse(client1 == client2);
- }
-}
[38/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java
new file mode 100644
index 0000000..d2d61a9
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java
@@ -0,0 +1,146 @@
+/**
+ * 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 org.apache.distributedlog.client.routing;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import com.twitter.finagle.Address;
+import com.twitter.finagle.Addresses;
+import com.twitter.finagle.addr.WeightedAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for {@link RoutingService}.
+ */
+@RunWith(Parameterized.class)
+public class TestRoutingService {
+
+ static final Logger LOG = LoggerFactory.getLogger(TestRoutingService.class);
+
+ @Parameterized.Parameters
+ public static Collection<Object[]> configs() {
+ ArrayList<Object[]> list = new ArrayList<Object[]>();
+ for (int i = 0; i <= 1; i++) {
+ for (int j = 0; j <= 1; j++) {
+ for (int k = 0; k <= 1; k++) {
+ list.add(new Boolean[] {i == 1, j == 1, k == 1});
+ }
+ }
+ }
+ return list;
+ }
+
+ private final boolean consistentHash;
+ private final boolean weightedAddresses;
+ private final boolean asyncResolution;
+
+ public TestRoutingService(boolean consistentHash, boolean weightedAddresses, boolean asyncResolution) {
+ this.consistentHash = consistentHash;
+ this.weightedAddresses = weightedAddresses;
+ this.asyncResolution = asyncResolution;
+ }
+
+ private List<Address> getAddresses(boolean weightedAddresses) {
+ ArrayList<Address> addresses = new ArrayList<Address>();
+ addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.1", 3181)));
+ addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.2", 3181)));
+ addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.3", 3181)));
+ addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.4", 3181)));
+ addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.5", 3181)));
+ addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.6", 3181)));
+ addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.7", 3181)));
+
+ if (weightedAddresses) {
+ ArrayList<Address> wAddresses = new ArrayList<Address>();
+ for (Address address: addresses) {
+ wAddresses.add(WeightedAddress.apply(address, 1.0));
+ }
+ return wAddresses;
+ } else {
+ return addresses;
+ }
+ }
+
+ private void testRoutingServiceHelper(boolean consistentHash,
+ boolean weightedAddresses,
+ boolean asyncResolution)
+ throws Exception {
+ ExecutorService executorService = null;
+ final List<Address> addresses = getAddresses(weightedAddresses);
+ final TestName name = new TestName();
+ RoutingService routingService;
+ if (consistentHash) {
+ routingService = ConsistentHashRoutingService.newBuilder()
+ .serverSet(new NameServerSet(name))
+ .resolveFromName(true)
+ .numReplicas(997)
+ .build();
+ } else {
+ routingService = ServerSetRoutingService.newServerSetRoutingServiceBuilder()
+ .serverSetWatcher(new TwitterServerSetWatcher(new NameServerSet(name), true)).build();
+ }
+
+ if (asyncResolution) {
+ executorService = Executors.newSingleThreadExecutor();
+ executorService.submit(new Runnable() {
+ @Override
+ public void run() {
+ name.changeAddrs(addresses);
+ }
+ });
+ } else {
+ name.changeAddrs(addresses);
+ }
+ routingService.startService();
+
+ HashSet<SocketAddress> mapping = new HashSet<SocketAddress>();
+
+ for (int i = 0; i < 1000; i++) {
+ for (int j = 0; j < 5; j++) {
+ String stream = "TestStream-" + i + "-" + j;
+ mapping.add(routingService.getHost(stream,
+ RoutingService.RoutingContext.of(new DefaultRegionResolver())));
+ }
+ }
+
+ assertEquals(mapping.size(), addresses.size());
+
+ if (null != executorService) {
+ executorService.shutdown();
+ }
+
+ }
+
+ @Test(timeout = 5000)
+ public void testRoutingService() throws Exception {
+ testRoutingServiceHelper(this.consistentHash, this.weightedAddresses, this.asyncResolution);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
new file mode 100644
index 0000000..ab0cb58
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
@@ -0,0 +1,105 @@
+/**
+ * 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 org.apache.distributedlog.client.speculative;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+
+import com.twitter.util.CountDownLatch;
+import com.twitter.util.Future;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test {@link TestDefaultSpeculativeRequestExecutionPolicy}.
+ */
+public class TestDefaultSpeculativeRequestExecutionPolicy {
+
+ @Test(timeout = 20000, expected = IllegalArgumentException.class)
+ public void testInvalidBackoffMultiplier() throws Exception {
+ new DefaultSpeculativeRequestExecutionPolicy(100, 200, -1);
+ }
+
+ @Test(timeout = 20000, expected = IllegalArgumentException.class)
+ public void testInvalidMaxSpeculativeTimeout() throws Exception {
+ new DefaultSpeculativeRequestExecutionPolicy(100, Integer.MAX_VALUE, 2);
+ }
+
+ @Test(timeout = 20000)
+ public void testSpeculativeRequests() throws Exception {
+ DefaultSpeculativeRequestExecutionPolicy policy =
+ new DefaultSpeculativeRequestExecutionPolicy(10, 10000, 2);
+ SpeculativeRequestExecutor executor = mock(SpeculativeRequestExecutor.class);
+
+ final AtomicInteger callCount = new AtomicInteger(0);
+ final CountDownLatch latch = new CountDownLatch(3);
+
+ Mockito.doAnswer(new Answer() {
+ @Override
+ public Object answer(InvocationOnMock invocation) throws Throwable {
+ try {
+ return Future.value(callCount.incrementAndGet() < 3);
+ } finally {
+ latch.countDown();
+ }
+ }
+ }).when(executor).issueSpeculativeRequest();
+
+ ScheduledExecutorService executorService =
+ Executors.newSingleThreadScheduledExecutor();
+ policy.initiateSpeculativeRequest(executorService, executor);
+
+ latch.await();
+
+ assertEquals(40, policy.getNextSpeculativeRequestTimeout());
+ }
+
+ @Test(timeout = 20000)
+ public void testSpeculativeRequestsWithMaxTimeout() throws Exception {
+ DefaultSpeculativeRequestExecutionPolicy policy =
+ new DefaultSpeculativeRequestExecutionPolicy(10, 15, 2);
+ SpeculativeRequestExecutor executor = mock(SpeculativeRequestExecutor.class);
+
+ final AtomicInteger callCount = new AtomicInteger(0);
+ final CountDownLatch latch = new CountDownLatch(3);
+
+ Mockito.doAnswer(new Answer() {
+ @Override
+ public Object answer(InvocationOnMock invocation) throws Throwable {
+ try {
+ return Future.value(callCount.incrementAndGet() < 3);
+ } finally {
+ latch.countDown();
+ }
+ }
+ }).when(executor).issueSpeculativeRequest();
+
+ ScheduledExecutorService executorService =
+ Executors.newSingleThreadScheduledExecutor();
+ policy.initiateSpeculativeRequest(executorService, executor);
+
+ latch.await();
+
+ assertEquals(15, policy.getNextSpeculativeRequestTimeout());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java
----------------------------------------------------------------------
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java b/distributedlog-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java
new file mode 100644
index 0000000..d2df9a5
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java
@@ -0,0 +1,49 @@
+/**
+ * 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 org.apache.distributedlog.service;
+
+import static org.junit.Assert.assertFalse;
+
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration;
+import org.junit.Test;
+
+/**
+ * Test Case of {@link org.apache.distributedlog.service.DistributedLogClientBuilder}.
+ */
+public class TestDistributedLogClientBuilder {
+
+ @Test(timeout = 60000)
+ public void testBuildClientsFromSameBuilder() throws Exception {
+ DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
+ .name("build-clients-from-same-builder")
+ .clientId(ClientId$.MODULE$.apply("test-builder"))
+ .finagleNameStr("inet!127.0.0.1:7001")
+ .streamNameRegex(".*")
+ .handshakeWithClientInfo(true)
+ .clientBuilder(ClientBuilder.get()
+ .hostConnectionLimit(1)
+ .connectTimeout(Duration.fromSeconds(1))
+ .tcpConnectTimeout(Duration.fromSeconds(1))
+ .requestTimeout(Duration.fromSeconds(10)));
+ DistributedLogClient client1 = builder.build();
+ DistributedLogClient client2 = builder.build();
+ assertFalse(client1 == client2);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/distributedlog-core/conf/log4j.properties b/distributedlog-core/conf/log4j.properties
index cafc888..38ab34d 100644
--- a/distributedlog-core/conf/log4j.properties
+++ b/distributedlog-core/conf/log4j.properties
@@ -32,11 +32,11 @@ log4j.logger.org.apache.zookeeper=INFO
log4j.logger.org.apache.bookkeeper=INFO
# redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.com.twitter.distributedlog.util.MonitoredFuturePool=INFO, Executors
-log4j.logger.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
+log4j.logger.org.apache.distributedlog.util.MonitoredFuturePool=INFO, Executors
+log4j.logger.org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
log4j.logger.org.apache.bookkeeper.util.SafeRunnable=INFO, Executors
-log4j.additivity.com.twitter.distributedlog.util.MonitoredFuturePool=false
-log4j.additivity.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
+log4j.additivity.org.apache.distributedlog.util.MonitoredFuturePool=false
+log4j.additivity.org.apache.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
log4j.additivity.org.apache.bookkeeper.util.SafeRunnable=false
log4j.appender.Executors=org.apache.log4j.RollingFileAppender
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/conf/zookeeper.conf.dynamic.template
----------------------------------------------------------------------
diff --git a/distributedlog-core/conf/zookeeper.conf.dynamic.template b/distributedlog-core/conf/zookeeper.conf.dynamic.template
index 4bda9f1..f4e35f5 100644
--- a/distributedlog-core/conf/zookeeper.conf.dynamic.template
+++ b/distributedlog-core/conf/zookeeper.conf.dynamic.template
@@ -1 +1 @@
-#/**# * Copyright 2007 The Apache Software Foundation# *# * 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.# */server.1=127.0.0.1:2710:3710:participant;0.0.0.0:2181
\ No newline at end of file
+#/**# * Copyright 2007 The Apache Software Foundation# *# * 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.# */server.1=127.0.0.1:2710:3710:participant;0.0.0.0:2181
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/pom.xml
----------------------------------------------------------------------
diff --git a/distributedlog-core/pom.xml b/distributedlog-core/pom.xml
index c5329aa..c4bfa8f 100644
--- a/distributedlog-core/pom.xml
+++ b/distributedlog-core/pom.xml
@@ -206,7 +206,7 @@
<properties>
<property>
<name>listener</name>
- <value>com.twitter.distributedlog.TimedOutTestsListener</value>
+ <value>org.apache.distributedlog.TimedOutTestsListener</value>
</property>
</properties>
</configuration>
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamReader.java
deleted file mode 100644
index 0f93bfe..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamReader.java
+++ /dev/null
@@ -1,198 +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.Preconditions;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class AppendOnlyStreamReader extends InputStream {
- static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamReader.class);
-
- private LogRecordWithInputStream currentLogRecord = null;
- private final DistributedLogManager dlm;
- private LogReader reader;
- private long currentPosition;
- private static final int SKIP_BUFFER_SIZE = 512;
-
- // Cache the input stream for a log record.
- private static class LogRecordWithInputStream {
- private final InputStream payloadStream;
- private final LogRecordWithDLSN logRecord;
-
- LogRecordWithInputStream(LogRecordWithDLSN logRecord) {
- Preconditions.checkNotNull(logRecord);
-
- LOG.debug("Got record dlsn = {}, txid = {}, len = {}",
- new Object[] {logRecord.getDlsn(), logRecord.getTransactionId(), logRecord.getPayload().length});
-
- this.logRecord = logRecord;
- this.payloadStream = logRecord.getPayLoadInputStream();
- }
-
- InputStream getPayLoadInputStream() {
- return payloadStream;
- }
-
- LogRecordWithDLSN getLogRecord() {
- return logRecord;
- }
-
- // The last txid of the log record is the position of the next byte in the stream.
- // Subtract length to get starting offset.
- long getOffset() {
- return logRecord.getTransactionId() - logRecord.getPayload().length;
- }
- }
-
- /**
- * Construct ledger input stream
- *
- * @param dlm the Distributed Log Manager to access the stream
- */
- AppendOnlyStreamReader(DistributedLogManager dlm)
- throws IOException {
- this.dlm = dlm;
- reader = dlm.getInputStream(0);
- currentPosition = 0;
- }
-
- /**
- * Get input stream representing next entry in the
- * ledger.
- *
- * @return input stream, or null if no more entries
- */
- private LogRecordWithInputStream nextLogRecord() throws IOException {
- return nextLogRecord(reader);
- }
-
- private static LogRecordWithInputStream nextLogRecord(LogReader reader) throws IOException {
- LogRecordWithDLSN record = reader.readNext(false);
-
- if (null != record) {
- return new LogRecordWithInputStream(record);
- } else {
- record = reader.readNext(false);
- if (null != record) {
- return new LogRecordWithInputStream(record);
- } else {
- LOG.debug("No record");
- return null;
- }
- }
- }
-
- @Override
- public int read() throws IOException {
- byte[] b = new byte[1];
- if (read(b, 0, 1) != 1) {
- return -1;
- } else {
- return b[0];
- }
- }
-
- @Override
- public int read(byte[] b, int off, int len) throws IOException {
- int read = 0;
- if (currentLogRecord == null) {
- currentLogRecord = nextLogRecord();
- if (currentLogRecord == null) {
- return read;
- }
- }
-
- while (read < len) {
- int thisread = currentLogRecord.getPayLoadInputStream().read(b, off + read, (len - read));
- if (thisread == -1) {
- currentLogRecord = nextLogRecord();
- if (currentLogRecord == null) {
- return read;
- }
- } else {
- LOG.debug("Offset saved = {}, persisted = {}",
- currentPosition, currentLogRecord.getLogRecord().getTransactionId());
- currentPosition += thisread;
- read += thisread;
- }
- }
- return read;
- }
-
- /**
- * Position the reader at the given offset. If we fail to skip to the desired position
- * and don't hit end of stream, return false.
- *
- * @throws com.twitter.distributedlog.exceptions.EndOfStreamException if we attempt to
- * skip past the end of the stream.
- */
- public boolean skipTo(long position) throws IOException {
-
- // No need to skip anywhere.
- if (position == position()) {
- return true;
- }
-
- LogReader skipReader = dlm.getInputStream(position);
- LogRecordWithInputStream logRecord = null;
- try {
- logRecord = nextLogRecord(skipReader);
- } catch (IOException ex) {
- skipReader.close();
- throw ex;
- }
-
- if (null == logRecord) {
- return false;
- }
-
- // We may end up with a reader positioned *before* the requested position if
- // we're near the tail and the writer is still active, or if the desired position
- // is not at a log record payload boundary.
- // Transaction ID gives us the starting position of the log record. Read ahead
- // if necessary.
- currentPosition = logRecord.getOffset();
- currentLogRecord = logRecord;
- LogReader oldReader = reader;
- reader = skipReader;
-
- // Close the oldreader after swapping AppendOnlyStreamReader state. Close may fail
- // and we need to make sure it leaves AppendOnlyStreamReader in a consistent state.
- oldReader.close();
-
- byte[] skipBuffer = new byte[SKIP_BUFFER_SIZE];
- while (currentPosition < position) {
- long bytesToRead = Math.min(position - currentPosition, SKIP_BUFFER_SIZE);
- long bytesRead = read(skipBuffer, 0, (int)bytesToRead);
- if (bytesRead < bytesToRead) {
- return false;
- }
- }
-
- return true;
- }
-
- public long position() {
- return currentPosition;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamWriter.java
deleted file mode 100644
index aa0aef9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamWriter.java
+++ /dev/null
@@ -1,107 +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.UnexpectedException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class AppendOnlyStreamWriter implements Closeable {
- static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamWriter.class);
-
- // Use a 1-length array to satisfy Java's inner class reference rules. Use primitive
- // type because synchronized block is needed anyway.
- final long[] syncPos = new long[1];
- BKAsyncLogWriter logWriter;
- long requestPos = 0;
-
- public AppendOnlyStreamWriter(BKAsyncLogWriter logWriter, long pos) {
- LOG.debug("initialize at position {}", pos);
- this.logWriter = logWriter;
- this.syncPos[0] = pos;
- this.requestPos = pos;
- }
-
- public Future<DLSN> write(byte[] data) {
- requestPos += data.length;
- Future<DLSN> writeResult = logWriter.write(new LogRecord(requestPos, data));
- return writeResult.addEventListener(new WriteCompleteListener(requestPos));
- }
-
- public void force(boolean metadata) throws IOException {
- long pos = 0;
- try {
- pos = Await.result(logWriter.flushAndCommit());
- } catch (IOException ioe) {
- throw ioe;
- } catch (Exception ex) {
- LOG.error("unexpected exception in AppendOnlyStreamWriter.force ", ex);
- throw new UnexpectedException("unexpected exception in AppendOnlyStreamWriter.force", ex);
- }
- synchronized (syncPos) {
- syncPos[0] = pos;
- }
- }
-
- public long position() {
- synchronized (syncPos) {
- return syncPos[0];
- }
- }
-
- @Override
- public void close() throws IOException {
- logWriter.closeAndComplete();
- }
-
- public void markEndOfStream() throws IOException {
- try {
- Await.result(logWriter.markEndOfStream());
- } catch (IOException ioe) {
- throw ioe;
- } catch (Exception ex) {
- throw new UnexpectedException("Mark end of stream hit unexpected exception", ex);
- }
- }
-
- class WriteCompleteListener implements FutureEventListener<DLSN> {
- private final long position;
- public WriteCompleteListener(long position) {
- this.position = position;
- }
- @Override
- public void onSuccess(DLSN response) {
- synchronized (syncPos) {
- if (position > syncPos[0]) {
- syncPos[0] = position;
- }
- }
- }
- @Override
- public void onFailure(Throwable cause) {
- // Handled at the layer above
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogReader.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogReader.java
deleted file mode 100644
index 8e07797..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogReader.java
+++ /dev/null
@@ -1,69 +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.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-public interface AsyncLogReader extends AsyncCloseable {
-
- /**
- * Get stream name that the reader reads from.
- *
- * @return stream name.
- */
- public String getStreamName();
-
- /**
- * Read the next record from the log stream
- *
- * @return A promise that when satisfied will contain the Log Record with its DLSN.
- */
- public Future<LogRecordWithDLSN> readNext();
-
- /**
- * Read next <i>numEntries</i> entries. The future is only satisfied with non-empty list
- * of entries. It doesn't block until returning exact <i>numEntries</i>. It is a best effort
- * call.
- *
- * @param numEntries
- * num entries
- * @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
- */
- public Future<List<LogRecordWithDLSN>> readBulk(int numEntries);
-
- /**
- * Read next <i>numEntries</i> entries in a given <i>waitTime</i>.
- * <p>
- * The future is satisfied when either reads <i>numEntries</i> entries or reaches <i>waitTime</i>.
- * The only exception is if there isn't any new entries written within <i>waitTime</i>, it would
- * wait until new entries are available.
- *
- * @param numEntries
- * max entries to return
- * @param waitTime
- * maximum wait time if there are entries already for read
- * @param timeUnit
- * wait time unit
- * @return A promise that when satisfied will contain a non-empty list of records with their DLSN.
- */
- public Future<List<LogRecordWithDLSN>> readBulk(int numEntries, long waitTime, TimeUnit timeUnit);
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogWriter.java
deleted file mode 100644
index e83e343..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogWriter.java
+++ /dev/null
@@ -1,70 +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.io.AsyncAbortable;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.io.Closeable;
-import java.util.List;
-
-public interface AsyncLogWriter extends AsyncCloseable, AsyncAbortable {
-
- /**
- * Get the last committed transaction id.
- *
- * @return last committed transaction id.
- */
- public long getLastTxId();
-
- /**
- * Write a log record to the stream.
- *
- * @param record single log record
- * @return A Future which contains a DLSN if the record was successfully written
- * or an exception if the write fails
- */
- public Future<DLSN> write(LogRecord record);
-
- /**
- * Write log records to the stream in bulk. Each future in the list represents the result of
- * one write operation. The size of the result list is equal to the size of the input list.
- * Buffers are written in order, and the list of result futures has the same order.
- *
- * @param record set of log records
- * @return A Future which contains a list of Future DLSNs if the record was successfully written
- * or an exception if the operation fails.
- */
- public Future<List<Future<DLSN>>> writeBulk(List<LogRecord> record);
-
- /**
- * Truncate the log until <i>dlsn</i>.
- *
- * @param dlsn
- * dlsn to truncate until.
- * @return A Future indicates whether the operation succeeds or not, or an exception
- * if the truncation fails.
- */
- public Future<Boolean> truncate(DLSN dlsn);
-
- /**
- * Get the name of the stream this writer writes data to
- */
- public String getStreamName();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncNotification.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncNotification.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncNotification.java
deleted file mode 100644
index bd71147..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncNotification.java
+++ /dev/null
@@ -1,32 +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;
-
-public interface AsyncNotification {
- /**
- * Triggered when the background activity encounters an exception
- *
- * @param reason the exception that encountered.
- */
- void notifyOnError(Throwable reason);
-
- /**
- * Triggered when the background activity completes an operation
- */
- void notifyOnOperationComplete();
-}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAbstractLogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAbstractLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAbstractLogWriter.java
deleted file mode 100644
index d1c28d7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAbstractLogWriter.java
+++ /dev/null
@@ -1,555 +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.annotations.VisibleForTesting;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.io.Abortable;
-import com.twitter.distributedlog.io.Abortables;
-import com.twitter.distributedlog.io.AsyncAbortable;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.PermitManager;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-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.util.List;
-import java.util.concurrent.TimeUnit;
-
-abstract class BKAbstractLogWriter implements Closeable, AsyncCloseable, Abortable, AsyncAbortable {
- static final Logger LOG = LoggerFactory.getLogger(BKAbstractLogWriter.class);
-
- protected final DistributedLogConfiguration conf;
- private final DynamicDistributedLogConfiguration dynConf;
- protected final BKDistributedLogManager bkDistributedLogManager;
-
- // States
- private Promise<Void> closePromise = null;
- private volatile boolean forceRolling = false;
- private boolean forceRecovery = false;
-
- // Truncation Related
- private Future<List<LogSegmentMetadata>> lastTruncationAttempt = null;
- @VisibleForTesting
- private Long minTimestampToKeepOverride = null;
-
- // Log Segment Writers
- protected BKLogSegmentWriter segmentWriter = null;
- protected Future<BKLogSegmentWriter> segmentWriterFuture = null;
- protected BKLogSegmentWriter allocatedSegmentWriter = null;
- protected BKLogWriteHandler writeHandler = null;
-
- BKAbstractLogWriter(DistributedLogConfiguration conf,
- DynamicDistributedLogConfiguration dynConf,
- BKDistributedLogManager bkdlm) {
- this.conf = conf;
- this.dynConf = dynConf;
- this.bkDistributedLogManager = bkdlm;
- LOG.debug("Initial retention period for {} : {}", bkdlm.getStreamName(),
- TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS));
- }
-
- // manage write handler
-
- synchronized protected BKLogWriteHandler getCachedWriteHandler() {
- return writeHandler;
- }
-
- protected BKLogWriteHandler getWriteHandler() throws IOException {
- BKLogWriteHandler writeHandler = createAndCacheWriteHandler();
- writeHandler.checkMetadataException();
- return writeHandler;
- }
-
- protected BKLogWriteHandler createAndCacheWriteHandler()
- throws IOException {
- synchronized (this) {
- if (writeHandler != null) {
- return writeHandler;
- }
- }
- // This code path will be executed when the handler is not set or has been closed
- // due to forceRecovery during testing
- BKLogWriteHandler newHandler =
- FutureUtils.result(bkDistributedLogManager.asyncCreateWriteHandler(false));
- boolean success = false;
- try {
- synchronized (this) {
- if (writeHandler == null) {
- writeHandler = newHandler;
- success = true;
- }
- return writeHandler;
- }
- } finally {
- if (!success) {
- newHandler.asyncAbort();
- }
- }
- }
-
- // manage log segment writers
-
- protected synchronized BKLogSegmentWriter getCachedLogWriter() {
- return segmentWriter;
- }
-
- protected synchronized Future<BKLogSegmentWriter> getCachedLogWriterFuture() {
- return segmentWriterFuture;
- }
-
- protected synchronized void cacheLogWriter(BKLogSegmentWriter logWriter) {
- this.segmentWriter = logWriter;
- this.segmentWriterFuture = Future.value(logWriter);
- }
-
- protected synchronized BKLogSegmentWriter removeCachedLogWriter() {
- try {
- return segmentWriter;
- } finally {
- segmentWriter = null;
- segmentWriterFuture = null;
- }
- }
-
- protected synchronized BKLogSegmentWriter getAllocatedLogWriter() {
- return allocatedSegmentWriter;
- }
-
- protected synchronized void cacheAllocatedLogWriter(BKLogSegmentWriter logWriter) {
- this.allocatedSegmentWriter = logWriter;
- }
-
- protected synchronized BKLogSegmentWriter removeAllocatedLogWriter() {
- try {
- return allocatedSegmentWriter;
- } finally {
- allocatedSegmentWriter = null;
- }
- }
-
- private Future<Void> asyncCloseAndComplete(boolean shouldThrow) {
- BKLogSegmentWriter segmentWriter = getCachedLogWriter();
- BKLogWriteHandler writeHandler = getCachedWriteHandler();
- if (null != segmentWriter && null != writeHandler) {
- cancelTruncation();
- Promise<Void> completePromise = new Promise<Void>();
- asyncCloseAndComplete(segmentWriter, writeHandler, completePromise, shouldThrow);
- return completePromise;
- } else {
- return closeNoThrow();
- }
- }
-
- private void asyncCloseAndComplete(final BKLogSegmentWriter segmentWriter,
- final BKLogWriteHandler writeHandler,
- final Promise<Void> completePromise,
- final boolean shouldThrow) {
- writeHandler.completeAndCloseLogSegment(segmentWriter)
- .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
- @Override
- public void onSuccess(LogSegmentMetadata segment) {
- removeCachedLogWriter();
- complete(null);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- LOG.error("Completing Log segments encountered exception", cause);
- complete(cause);
- }
-
- private void complete(final Throwable cause) {
- closeNoThrow().ensure(new AbstractFunction0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- if (null != cause && shouldThrow) {
- FutureUtils.setException(completePromise, cause);
- } else {
- FutureUtils.setValue(completePromise, null);
- }
- return BoxedUnit.UNIT;
- }
- });
- }
- });
- }
-
- @VisibleForTesting
- void closeAndComplete() throws IOException {
- FutureUtils.result(asyncCloseAndComplete(true));
- }
-
- protected Future<Void> asyncCloseAndComplete() {
- return asyncCloseAndComplete(true);
- }
-
- @Override
- public void close() throws IOException {
- FutureUtils.result(asyncClose());
- }
-
- @Override
- public Future<Void> asyncClose() {
- return asyncCloseAndComplete(false);
- }
-
- /**
- * Close the writer and release all the underlying resources
- */
- protected Future<Void> closeNoThrow() {
- Promise<Void> closeFuture;
- synchronized (this) {
- if (null != closePromise) {
- return closePromise;
- }
- closeFuture = closePromise = new Promise<Void>();
- }
- cancelTruncation();
- Utils.closeSequence(bkDistributedLogManager.getScheduler(),
- true, /** ignore close errors **/
- getCachedLogWriter(),
- getAllocatedLogWriter(),
- getCachedWriteHandler()
- ).proxyTo(closeFuture);
- return closeFuture;
- }
-
- @Override
- public void abort() throws IOException {
- FutureUtils.result(asyncAbort());
- }
-
- @Override
- public Future<Void> asyncAbort() {
- Promise<Void> closeFuture;
- synchronized (this) {
- if (null != closePromise) {
- return closePromise;
- }
- closeFuture = closePromise = new Promise<Void>();
- }
- cancelTruncation();
- Abortables.abortSequence(bkDistributedLogManager.getScheduler(),
- getCachedLogWriter(),
- getAllocatedLogWriter(),
- getCachedWriteHandler()).proxyTo(closeFuture);
- return closeFuture;
- }
-
- // used by sync writer
- protected BKLogSegmentWriter getLedgerWriter(final long startTxId,
- final boolean allowMaxTxID)
- throws IOException {
- Future<BKLogSegmentWriter> logSegmentWriterFuture = asyncGetLedgerWriter(true);
- BKLogSegmentWriter logSegmentWriter = null;
- if (null != logSegmentWriterFuture) {
- logSegmentWriter = FutureUtils.result(logSegmentWriterFuture);
- }
- if (null == logSegmentWriter || (shouldStartNewSegment(logSegmentWriter) || forceRolling)) {
- logSegmentWriter = FutureUtils.result(rollLogSegmentIfNecessary(
- logSegmentWriter, startTxId, true /* bestEffort */, allowMaxTxID));
- }
- return logSegmentWriter;
- }
-
- // used by async writer
- synchronized protected Future<BKLogSegmentWriter> asyncGetLedgerWriter(boolean resetOnError) {
- final BKLogSegmentWriter ledgerWriter = getCachedLogWriter();
- Future<BKLogSegmentWriter> ledgerWriterFuture = getCachedLogWriterFuture();
- if (null == ledgerWriterFuture || null == ledgerWriter) {
- return null;
- }
-
- // Handle the case where the last call to write actually caused an error in the log
- if ((ledgerWriter.isLogSegmentInError() || forceRecovery) && resetOnError) {
- // Close the ledger writer so that we will recover and start a new log segment
- Future<Void> closeFuture;
- if (ledgerWriter.isLogSegmentInError()) {
- closeFuture = ledgerWriter.asyncAbort();
- } else {
- closeFuture = ledgerWriter.asyncClose();
- }
- return closeFuture.flatMap(
- new AbstractFunction1<Void, Future<BKLogSegmentWriter>>() {
- @Override
- public Future<BKLogSegmentWriter> apply(Void result) {
- removeCachedLogWriter();
-
- if (ledgerWriter.isLogSegmentInError()) {
- return Future.value(null);
- }
-
- BKLogWriteHandler writeHandler;
- try {
- writeHandler = getWriteHandler();
- } catch (IOException e) {
- return Future.exception(e);
- }
- if (null != writeHandler && forceRecovery) {
- return writeHandler.completeAndCloseLogSegment(ledgerWriter)
- .map(new AbstractFunction1<LogSegmentMetadata, BKLogSegmentWriter>() {
- @Override
- public BKLogSegmentWriter apply(LogSegmentMetadata completedLogSegment) {
- return null;
- }
- });
- } else {
- return Future.value(null);
- }
- }
- });
- } else {
- return ledgerWriterFuture;
- }
- }
-
- boolean shouldStartNewSegment(BKLogSegmentWriter ledgerWriter) throws IOException {
- BKLogWriteHandler writeHandler = getWriteHandler();
- return null == ledgerWriter || writeHandler.shouldStartNewSegment(ledgerWriter) || forceRolling;
- }
-
- private void truncateLogSegmentsIfNecessary(BKLogWriteHandler writeHandler) {
- boolean truncationEnabled = false;
-
- long minTimestampToKeep = 0;
-
- long retentionPeriodInMillis = TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS);
- if (retentionPeriodInMillis > 0) {
- minTimestampToKeep = Utils.nowInMillis() - retentionPeriodInMillis;
- truncationEnabled = true;
- }
-
- if (null != minTimestampToKeepOverride) {
- minTimestampToKeep = minTimestampToKeepOverride;
- truncationEnabled = true;
- }
-
- // skip scheduling if there is task that's already running
- //
- synchronized (this) {
- if (truncationEnabled && ((lastTruncationAttempt == null) || lastTruncationAttempt.isDefined())) {
- lastTruncationAttempt = writeHandler.purgeLogSegmentsOlderThanTimestamp(minTimestampToKeep);
- }
- }
- }
-
- private Future<BKLogSegmentWriter> asyncStartNewLogSegment(final BKLogWriteHandler writeHandler,
- final long startTxId,
- final boolean allowMaxTxID) {
- return writeHandler.recoverIncompleteLogSegments()
- .flatMap(new AbstractFunction1<Long, Future<BKLogSegmentWriter>>() {
- @Override
- public Future<BKLogSegmentWriter> apply(Long lastTxId) {
- return writeHandler.asyncStartLogSegment(startTxId, false, allowMaxTxID)
- .onSuccess(new AbstractFunction1<BKLogSegmentWriter, BoxedUnit>() {
- @Override
- public BoxedUnit apply(BKLogSegmentWriter newSegmentWriter) {
- cacheLogWriter(newSegmentWriter);
- return BoxedUnit.UNIT;
- }
- });
- }
- });
- }
-
- private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOneWithPermit(
- final BKLogSegmentWriter oldSegmentWriter,
- final BKLogWriteHandler writeHandler,
- final long startTxId,
- final boolean bestEffort,
- final boolean allowMaxTxID) {
- final PermitManager.Permit switchPermit = bkDistributedLogManager.getLogSegmentRollingPermitManager().acquirePermit();
- if (switchPermit.isAllowed()) {
- return closeOldLogSegmentAndStartNewOne(
- oldSegmentWriter,
- writeHandler,
- startTxId,
- bestEffort,
- allowMaxTxID
- ).rescue(new Function<Throwable, Future<BKLogSegmentWriter>>() {
- @Override
- public Future<BKLogSegmentWriter> apply(Throwable cause) {
- if (cause instanceof LockingException) {
- LOG.warn("We lost lock during completeAndClose log segment for {}. Disable ledger rolling until it is recovered : ",
- writeHandler.getFullyQualifiedName(), cause);
- bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
- return Future.value(oldSegmentWriter);
- } else if (cause instanceof ZKException) {
- ZKException zke = (ZKException) cause;
- if (ZKException.isRetryableZKException(zke)) {
- LOG.warn("Encountered zookeeper connection issues during completeAndClose log segment for {}." +
- " Disable ledger rolling until it is recovered : {}", writeHandler.getFullyQualifiedName(),
- zke.getKeeperExceptionCode());
- bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
- return Future.value(oldSegmentWriter);
- }
- }
- return Future.exception(cause);
- }
- }).ensure(new AbstractFunction0<BoxedUnit>() {
- @Override
- public BoxedUnit apply() {
- bkDistributedLogManager.getLogSegmentRollingPermitManager()
- .releasePermit(switchPermit);
- return BoxedUnit.UNIT;
- }
- });
- } else {
- bkDistributedLogManager.getLogSegmentRollingPermitManager().releasePermit(switchPermit);
- return Future.value(oldSegmentWriter);
- }
- }
-
- private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOne(
- final BKLogSegmentWriter oldSegmentWriter,
- final BKLogWriteHandler writeHandler,
- final long startTxId,
- final boolean bestEffort,
- final boolean allowMaxTxID) {
- // we switch only when we could allocate a new log segment.
- BKLogSegmentWriter newSegmentWriter = getAllocatedLogWriter();
- if (null == newSegmentWriter) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Allocating a new log segment from {} for {}.", startTxId,
- writeHandler.getFullyQualifiedName());
- }
- return writeHandler.asyncStartLogSegment(startTxId, bestEffort, allowMaxTxID)
- .flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<BKLogSegmentWriter>>() {
- @Override
- public Future<BKLogSegmentWriter> apply(BKLogSegmentWriter newSegmentWriter) {
- if (null == newSegmentWriter) {
- if (bestEffort) {
- return Future.value(oldSegmentWriter);
- } else {
- return Future.exception(
- new UnexpectedException("StartLogSegment returns null for bestEffort rolling"));
- }
- }
- cacheAllocatedLogWriter(newSegmentWriter);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Allocated a new log segment from {} for {}.", startTxId,
- writeHandler.getFullyQualifiedName());
- }
- return completeOldSegmentAndCacheNewLogSegmentWriter(oldSegmentWriter, newSegmentWriter);
- }
- });
- } else {
- return completeOldSegmentAndCacheNewLogSegmentWriter(oldSegmentWriter, newSegmentWriter);
- }
- }
-
- private Future<BKLogSegmentWriter> completeOldSegmentAndCacheNewLogSegmentWriter(
- BKLogSegmentWriter oldSegmentWriter,
- final BKLogSegmentWriter newSegmentWriter) {
- final Promise<BKLogSegmentWriter> completePromise = new Promise<BKLogSegmentWriter>();
- // complete the old log segment
- writeHandler.completeAndCloseLogSegment(oldSegmentWriter)
- .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
-
- @Override
- public void onSuccess(LogSegmentMetadata value) {
- cacheLogWriter(newSegmentWriter);
- removeAllocatedLogWriter();
- FutureUtils.setValue(completePromise, newSegmentWriter);
- }
-
- @Override
- public void onFailure(Throwable cause) {
- FutureUtils.setException(completePromise, cause);
- }
- });
- return completePromise;
- }
-
- synchronized protected Future<BKLogSegmentWriter> rollLogSegmentIfNecessary(
- final BKLogSegmentWriter segmentWriter,
- long startTxId,
- boolean bestEffort,
- boolean allowMaxTxID) {
- final BKLogWriteHandler writeHandler;
- try {
- writeHandler = getWriteHandler();
- } catch (IOException e) {
- return Future.exception(e);
- }
- Future<BKLogSegmentWriter> rollPromise;
- if (null != segmentWriter && (writeHandler.shouldStartNewSegment(segmentWriter) || forceRolling)) {
- rollPromise = closeOldLogSegmentAndStartNewOneWithPermit(
- segmentWriter, writeHandler, startTxId, bestEffort, allowMaxTxID);
- } else if (null == segmentWriter) {
- rollPromise = asyncStartNewLogSegment(writeHandler, startTxId, allowMaxTxID);
- } else {
- rollPromise = Future.value(segmentWriter);
- }
- return rollPromise.map(new AbstractFunction1<BKLogSegmentWriter, BKLogSegmentWriter>() {
- @Override
- public BKLogSegmentWriter apply(BKLogSegmentWriter newSegmentWriter) {
- if (segmentWriter == newSegmentWriter) {
- return newSegmentWriter;
- }
- truncateLogSegmentsIfNecessary(writeHandler);
- return newSegmentWriter;
- }
- });
- }
-
- protected synchronized void checkClosedOrInError(String operation) throws AlreadyClosedException {
- if (null != closePromise) {
- LOG.error("Executing " + operation + " on already closed Log Writer");
- throw new AlreadyClosedException("Executing " + operation + " on already closed Log Writer");
- }
- }
-
- @VisibleForTesting
- public void setForceRolling(boolean forceRolling) {
- this.forceRolling = forceRolling;
- }
-
- @VisibleForTesting
- public synchronized void overRideMinTimeStampToKeep(Long minTimestampToKeepOverride) {
- this.minTimestampToKeepOverride = minTimestampToKeepOverride;
- }
-
- protected synchronized void cancelTruncation() {
- if (null != lastTruncationAttempt) {
- FutureUtils.cancel(lastTruncationAttempt);
- lastTruncationAttempt = null;
- }
- }
-
- @VisibleForTesting
- public synchronized void setForceRecovery(boolean forceRecovery) {
- this.forceRecovery = forceRecovery;
- }
-
-}
[04/51] [partial] incubator-distributedlog git commit: DL-4:
Repackage the source under apache namespace
Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
new file mode 100644
index 0000000..c5050ec
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
@@ -0,0 +1,1125 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Comparator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
+import org.apache.distributedlog.exceptions.UnsupportedMetadataVersionException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.Utils;
+import com.twitter.util.Future;
+import com.twitter.util.Promise;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Utility class for storing the metadata associated
+ * with a single edit log segment, stored in a single ledger
+ */
+public class LogSegmentMetadata {
+ static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadata.class);
+
+ public static enum LogSegmentMetadataVersion {
+ VERSION_INVALID(0),
+ VERSION_V1_ORIGINAL(1),
+ VERSION_V2_LEDGER_SEQNO(2),
+ VERSION_V3_MIN_ACTIVE_DLSN(3),
+ VERSION_V4_ENVELOPED_ENTRIES(4),
+ VERSION_V5_SEQUENCE_ID(5);
+
+ public final int value;
+
+ private LogSegmentMetadataVersion(int value) {
+ this.value = value;
+ }
+
+ public static LogSegmentMetadataVersion of(int version) {
+ switch (version) {
+ case 5:
+ return VERSION_V5_SEQUENCE_ID;
+ case 4:
+ return VERSION_V4_ENVELOPED_ENTRIES;
+ case 3:
+ return VERSION_V3_MIN_ACTIVE_DLSN;
+ case 2:
+ return VERSION_V2_LEDGER_SEQNO;
+ case 1:
+ return VERSION_V1_ORIGINAL;
+ case 0:
+ return VERSION_INVALID;
+ default:
+ throw new IllegalArgumentException("unknown version " + version);
+ }
+ }
+ }
+
+ public static enum TruncationStatus {
+ ACTIVE (0), PARTIALLY_TRUNCATED(1), TRUNCATED (2);
+ private final int value;
+
+ private TruncationStatus(int value) {
+ this.value = value;
+ }
+ }
+
+ public static class LogSegmentMetadataBuilder {
+ protected String zkPath;
+ protected long logSegmentId;
+ protected LogSegmentMetadataVersion version;
+ protected long firstTxId;
+ protected int regionId;
+ protected long status;
+ protected long lastTxId;
+ protected long completionTime;
+ protected int recordCount;
+ protected long logSegmentSequenceNo;
+ protected long lastEntryId;
+ protected long lastSlotId;
+ protected long minActiveEntryId;
+ protected long minActiveSlotId;
+ protected long startSequenceId;
+ protected boolean inprogress;
+
+ // This is a derived attribute.
+ // Since we overwrite the original version with the target version, information that is
+ // derived from the original version (e.g. does it support enveloping of entries)
+ // is lost while parsing.
+ // NOTE: This value is not stored in the Metadata store.
+ protected boolean envelopeEntries = false;
+
+ LogSegmentMetadataBuilder(String zkPath,
+ LogSegmentMetadataVersion version,
+ long logSegmentId,
+ long firstTxId) {
+ initialize();
+ this.zkPath = zkPath;
+ this.version = version;
+ this.logSegmentId = logSegmentId;
+ this.firstTxId = firstTxId;
+ }
+
+ LogSegmentMetadataBuilder(String zkPath,
+ int version,
+ long logSegmentId,
+ long firstTxId) {
+ this(zkPath, LogSegmentMetadataVersion.values()[version], logSegmentId, firstTxId);
+ }
+
+ private void initialize() {
+ regionId = DistributedLogConstants.LOCAL_REGION_ID;
+ status = DistributedLogConstants.LOGSEGMENT_DEFAULT_STATUS;
+ lastTxId = DistributedLogConstants.INVALID_TXID;
+ completionTime = 0;
+ recordCount = 0;
+ lastEntryId = -1;
+ lastSlotId = -1;
+ minActiveEntryId = 0;
+ minActiveSlotId = 0;
+ startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
+ inprogress = true;
+ }
+
+ LogSegmentMetadataBuilder setRegionId(int regionId) {
+ this.regionId = regionId;
+ return this;
+ }
+
+ LogSegmentMetadataBuilder setStatus(long status) {
+ this.status = status;
+ return this;
+ }
+
+ public LogSegmentMetadataBuilder setLastTxId(long lastTxId) {
+ this.lastTxId = lastTxId;
+ return this;
+ }
+
+ public LogSegmentMetadataBuilder setCompletionTime(long completionTime) {
+ this.completionTime = completionTime;
+ return this;
+ }
+
+ public LogSegmentMetadataBuilder setRecordCount(int recordCount) {
+ this.recordCount = recordCount;
+ return this;
+ }
+
+ public LogSegmentMetadataBuilder setRecordCount(LogRecord record) {
+ this.recordCount = record.getLastPositionWithinLogSegment();
+ return this;
+ }
+
+ public LogSegmentMetadataBuilder setInprogress(boolean inprogress) {
+ this.inprogress = inprogress;
+ return this;
+ }
+
+ LogSegmentMetadataBuilder setLogSegmentSequenceNo(long logSegmentSequenceNo) {
+ this.logSegmentSequenceNo = logSegmentSequenceNo;
+ return this;
+ }
+
+ public LogSegmentMetadataBuilder setLastEntryId(long lastEntryId) {
+ this.lastEntryId = lastEntryId;
+ return this;
+ }
+
+ LogSegmentMetadataBuilder setLastSlotId(long lastSlotId) {
+ this.lastSlotId = lastSlotId;
+ return this;
+ }
+
+ LogSegmentMetadataBuilder setEnvelopeEntries(boolean envelopeEntries) {
+ this.envelopeEntries = envelopeEntries;
+ return this;
+ }
+
+ LogSegmentMetadataBuilder setMinActiveEntryId(long minActiveEntryId) {
+ this.minActiveEntryId = minActiveEntryId;
+ return this;
+ }
+
+ LogSegmentMetadataBuilder setMinActiveSlotId(long minActiveSlotId) {
+ this.minActiveSlotId = minActiveSlotId;
+ return this;
+ }
+
+ LogSegmentMetadataBuilder setStartSequenceId(long startSequenceId) {
+ this.startSequenceId = startSequenceId;
+ return this;
+ }
+
+ public LogSegmentMetadata build() {
+ return new LogSegmentMetadata(
+ zkPath,
+ version,
+ logSegmentId,
+ firstTxId,
+ lastTxId,
+ completionTime,
+ inprogress,
+ recordCount,
+ logSegmentSequenceNo,
+ lastEntryId,
+ lastSlotId,
+ regionId,
+ status,
+ minActiveEntryId,
+ minActiveSlotId,
+ startSequenceId,
+ envelopeEntries
+ );
+ }
+
+ }
+
+ /**
+ * Mutator to mutate the metadata of a log segment. This mutator is going to create
+ * a new instance of the log segment metadata without changing the existing one.
+ */
+ public static class Mutator extends LogSegmentMetadataBuilder {
+
+ Mutator(LogSegmentMetadata original) {
+ super(original.getZkPath(), original.getVersion(), original.getLogSegmentId(), original.getFirstTxId());
+ this.inprogress = original.isInProgress();
+ this.logSegmentSequenceNo = original.getLogSegmentSequenceNumber();
+ this.lastEntryId = original.getLastEntryId();
+ this.lastSlotId = original.getLastSlotId();
+ this.lastTxId = original.getLastTxId();
+ this.completionTime = original.getCompletionTime();
+ this.recordCount = original.getRecordCount();
+ this.regionId = original.getRegionId();
+ this.status = original.getStatus();
+ this.minActiveEntryId = original.getMinActiveDLSN().getEntryId();
+ this.minActiveSlotId = original.getMinActiveDLSN().getSlotId();
+ this.startSequenceId = original.getStartSequenceId();
+ this.envelopeEntries = original.getEnvelopeEntries();
+ }
+
+ @VisibleForTesting
+ public Mutator setVersion(LogSegmentMetadataVersion version) {
+ this.version = version;
+ return this;
+ }
+
+ public Mutator setLogSegmentSequenceNumber(long seqNo) {
+ this.logSegmentSequenceNo = seqNo;
+ return this;
+ }
+
+ public Mutator setZkPath(String zkPath) {
+ this.zkPath = zkPath;
+ return this;
+ }
+
+ public Mutator setLastDLSN(DLSN dlsn) {
+ this.logSegmentSequenceNo = dlsn.getLogSegmentSequenceNo();
+ this.lastEntryId = dlsn.getEntryId();
+ this.lastSlotId = dlsn.getSlotId();
+ return this;
+ }
+
+ public Mutator setMinActiveDLSN(DLSN dlsn) {
+ if (this.logSegmentSequenceNo != dlsn.getLogSegmentSequenceNo()) {
+ throw new IllegalArgumentException("Updating minDLSN in an incorrect log segment");
+ }
+ this.minActiveEntryId = dlsn.getEntryId();
+ this.minActiveSlotId = dlsn.getSlotId();
+ return this;
+ }
+
+ public Mutator setTruncationStatus(TruncationStatus truncationStatus) {
+ status &= ~METADATA_TRUNCATION_STATUS_MASK;
+ status |= (truncationStatus.value & METADATA_TRUNCATION_STATUS_MASK);
+ return this;
+ }
+
+ public Mutator setStartSequenceId(long startSequenceId) {
+ this.startSequenceId = startSequenceId;
+ return this;
+ }
+ }
+
+ private final String zkPath;
+ private final long logSegmentId;
+ private final LogSegmentMetadataVersion version;
+ private final long firstTxId;
+ private final int regionId;
+ private final long status;
+ private final long lastTxId;
+ private final long completionTime;
+ private final int recordCount;
+ private final DLSN lastDLSN;
+ private final DLSN minActiveDLSN;
+ private final long startSequenceId;
+ private final boolean inprogress;
+ // This is a derived attribute.
+ // Since we overwrite the original version with the target version, information that is
+ // derived from the original version (e.g. does it support enveloping of entries)
+ // is lost while parsing.
+ // NOTE: This value is not stored in the Metadata store.
+ private final boolean envelopeEntries;
+
+ public static final Comparator<LogSegmentMetadata> COMPARATOR
+ = new Comparator<LogSegmentMetadata>() {
+
+ public int compare(LogSegmentMetadata o1,
+ LogSegmentMetadata o2) {
+ if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
+ (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
+ if (o1.firstTxId < o2.firstTxId) {
+ return -1;
+ } else if (o1.firstTxId == o2.firstTxId) {
+ return 0;
+ } else {
+ return 1;
+ }
+ } else {
+ if (o1.getLogSegmentSequenceNumber() < o2.getLogSegmentSequenceNumber()) {
+ return -1;
+ } else if (o1.getLogSegmentSequenceNumber() == o2.getLogSegmentSequenceNumber()) {
+ // make sure we won't move over inprogress log segment if it still presents in the list
+ if (o1.isInProgress() && !o2.isInProgress()) {
+ return -1;
+ } else if (!o1.isInProgress() && o2.isInProgress()) {
+ return 1;
+ } else {
+ return 0;
+ }
+ } else {
+ return 1;
+ }
+ }
+
+
+ }
+ };
+
+ public static final Comparator<LogSegmentMetadata> DESC_COMPARATOR
+ = new Comparator<LogSegmentMetadata>() {
+ public int compare(LogSegmentMetadata o1,
+ LogSegmentMetadata o2) {
+ if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
+ (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
+ if (o1.firstTxId > o2.firstTxId) {
+ return -1;
+ } else if (o1.firstTxId == o2.firstTxId) {
+ return 0;
+ } else {
+ return 1;
+ }
+ } else {
+ if (o1.getLogSegmentSequenceNumber() > o2.getLogSegmentSequenceNumber()) {
+ return -1;
+ } else if (o1.getLogSegmentSequenceNumber() == o2.getLogSegmentSequenceNumber()) {
+ // make sure we won't move over inprogress log segment if it still presents in the list
+ if (o1.isInProgress() && !o2.isInProgress()) {
+ return 1;
+ } else if (!o1.isInProgress() && o2.isInProgress()) {
+ return -1;
+ } else {
+ return 0;
+ }
+ } else {
+ return 1;
+ }
+ }
+ }
+ };
+
+ public static final int LEDGER_METADATA_CURRENT_LAYOUT_VERSION =
+ LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
+
+ public static final int LEDGER_METADATA_OLDEST_SUPPORTED_VERSION =
+ LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value;
+
+ static final int LOGRECORD_COUNT_SHIFT = 32;
+ static final long LOGRECORD_COUNT_MASK = 0xffffffff00000000L;
+ static final int REGION_SHIFT = 28;
+ static final long MAX_REGION_ID = 0xfL;
+ static final long REGION_MASK = 0x00000000f0000000L;
+ static final int STATUS_BITS_SHIFT = 8;
+ static final long STATUS_BITS_MASK = 0x000000000000ff00L;
+ static final long UNUSED_BITS_MASK = 0x000000000fff0000L;
+ static final long METADATA_VERSION_MASK = 0x00000000000000ffL;
+
+ //Metadata status bits
+ static final long METADATA_TRUNCATION_STATUS_MASK = 0x3L;
+ static final long METADATA_STATUS_BIT_MAX = 0xffL;
+
+ private LogSegmentMetadata(String zkPath,
+ LogSegmentMetadataVersion version,
+ long logSegmentId,
+ long firstTxId,
+ long lastTxId,
+ long completionTime,
+ boolean inprogress,
+ int recordCount,
+ long logSegmentSequenceNumber,
+ long lastEntryId,
+ long lastSlotId,
+ int regionId,
+ long status,
+ long minActiveEntryId,
+ long minActiveSlotId,
+ long startSequenceId,
+ boolean envelopeEntries) {
+ this.zkPath = zkPath;
+ this.logSegmentId = logSegmentId;
+ this.version = version;
+ this.firstTxId = firstTxId;
+ this.lastTxId = lastTxId;
+ this.inprogress = inprogress;
+ this.completionTime = completionTime;
+ this.recordCount = recordCount;
+ this.lastDLSN = new DLSN(logSegmentSequenceNumber, lastEntryId, lastSlotId);
+ this.minActiveDLSN = new DLSN(logSegmentSequenceNumber, minActiveEntryId, minActiveSlotId);
+ this.startSequenceId = startSequenceId;
+ this.regionId = regionId;
+ this.status = status;
+ this.envelopeEntries = envelopeEntries;
+ }
+
+ public String getZkPath() {
+ return zkPath;
+ }
+
+ public String getZNodeName() {
+ return new File(zkPath).getName();
+ }
+
+ public long getFirstTxId() {
+ return firstTxId;
+ }
+
+ public long getLastTxId() {
+ return lastTxId;
+ }
+
+ public long getCompletionTime() {
+ return completionTime;
+ }
+
+ public long getLogSegmentId() {
+ return logSegmentId;
+ }
+
+ public long getLogSegmentSequenceNumber() {
+ return lastDLSN.getLogSegmentSequenceNo();
+ }
+
+ public int getVersion() {
+ return version.value;
+ }
+
+ public boolean getEnvelopeEntries() {
+ return envelopeEntries;
+ }
+
+ public long getLastEntryId() {
+ return lastDLSN.getEntryId();
+ }
+
+ long getStatus() {
+ return status;
+ }
+
+ public long getStartSequenceId() {
+ // generate negative sequence id for log segments that created <= v4
+ return supportsSequenceId() && startSequenceId != DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ?
+ startSequenceId : Long.MIN_VALUE + (getLogSegmentSequenceNumber() << 32L);
+ }
+
+ public boolean isTruncated() {
+ return ((status & METADATA_TRUNCATION_STATUS_MASK)
+ == TruncationStatus.TRUNCATED.value);
+ }
+
+ public boolean isPartiallyTruncated() {
+ return ((status & METADATA_TRUNCATION_STATUS_MASK)
+ == TruncationStatus.PARTIALLY_TRUNCATED.value);
+ }
+
+ public boolean isNonTruncated() {
+ return ((status & METADATA_TRUNCATION_STATUS_MASK)
+ == TruncationStatus.ACTIVE.value);
+ }
+
+ public long getLastSlotId() {
+ return lastDLSN.getSlotId();
+ }
+
+ public DLSN getLastDLSN() {
+ return lastDLSN;
+ }
+
+ public DLSN getMinActiveDLSN() {
+ return minActiveDLSN;
+ }
+
+ public DLSN getFirstDLSN() {
+ return new DLSN(getLogSegmentSequenceNumber(), 0, 0);
+ }
+
+ public int getRecordCount() {
+ return recordCount;
+ }
+
+ public int getRegionId() {
+ return regionId;
+ }
+
+ public boolean isInProgress() {
+ return this.inprogress;
+ }
+
+ @VisibleForTesting
+ public boolean isDLSNinThisSegment(DLSN dlsn) {
+ return dlsn.getLogSegmentSequenceNo() == getLogSegmentSequenceNumber();
+ }
+
+ @VisibleForTesting
+ public boolean isRecordPositionWithinSegmentScope(LogRecord record) {
+ return record.getLastPositionWithinLogSegment() <= getRecordCount();
+ }
+
+ @VisibleForTesting
+ public boolean isRecordLastPositioninThisSegment(LogRecord record) {
+ return record.getLastPositionWithinLogSegment() == getRecordCount();
+ }
+
+ /**
+ * complete current log segment. A new log segment metadata instance will be returned.
+ *
+ * @param zkPath
+ * zk path for the completed log segment.
+ * @param newLastTxId
+ * last tx id
+ * @param recordCount
+ * record count
+ * @param lastEntryId
+ * last entry id
+ * @param lastSlotId
+ * last slot id
+ * @return completed log segment.
+ */
+ LogSegmentMetadata completeLogSegment(String zkPath,
+ long newLastTxId,
+ int recordCount,
+ long lastEntryId,
+ long lastSlotId,
+ long startSequenceId) {
+ assert this.lastTxId == DistributedLogConstants.INVALID_TXID;
+
+ return new Mutator(this)
+ .setZkPath(zkPath)
+ .setLastDLSN(new DLSN(this.lastDLSN.getLogSegmentSequenceNo(), lastEntryId, lastSlotId))
+ .setLastTxId(newLastTxId)
+ .setInprogress(false)
+ .setCompletionTime(Utils.nowInMillis())
+ .setRecordCount(recordCount)
+ .setStartSequenceId(startSequenceId)
+ .build();
+ }
+
+ public static Future<LogSegmentMetadata> read(ZooKeeperClient zkc, String path) {
+ return read(zkc, path, false);
+ }
+
+ public static Future<LogSegmentMetadata> read(ZooKeeperClient zkc, String path, final boolean skipMinVersionCheck) {
+ final Promise<LogSegmentMetadata> result = new Promise<LogSegmentMetadata>();
+ try {
+ zkc.get().getData(path, false, new AsyncCallback.DataCallback() {
+ @Override
+ public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
+ if (KeeperException.Code.OK.intValue() != rc) {
+ if (KeeperException.Code.NONODE.intValue() == rc) {
+ FutureUtils.setException(result, new LogSegmentNotFoundException(path));
+ } else {
+ FutureUtils.setException(result,
+ new ZKException("Failed to read log segment metadata from " + path,
+ KeeperException.Code.get(rc)));
+ }
+ return;
+ }
+ try {
+ LogSegmentMetadata metadata = parseData(path, data, skipMinVersionCheck);
+ FutureUtils.setValue(result, metadata);
+ } catch (IOException ie) {
+ LOG.error("Error on parsing log segment metadata from {} : ", path, ie);
+ result.setException(ie);
+ }
+ }
+ }, null);
+ } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+ result.setException(FutureUtils.zkException(e, path));
+ } catch (InterruptedException e) {
+ result.setException(FutureUtils.zkException(e, path));
+ }
+ return result;
+ }
+
+ static LogSegmentMetadata parseDataV1(String path, byte[] data, String[] parts)
+ throws IOException {
+ long versionStatusCount = Long.parseLong(parts[0]);
+
+ long version = versionStatusCount & METADATA_VERSION_MASK;
+ assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
+ assert (1 == version);
+
+ LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V1_ORIGINAL;
+
+ int regionId = (int)(versionStatusCount & REGION_MASK) >> REGION_SHIFT;
+ assert (regionId >= 0 && regionId <= 0xf);
+
+ long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
+ assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
+
+ if (parts.length == 3) {
+ long logSegmentId = Long.parseLong(parts[1]);
+ long txId = Long.parseLong(parts[2]);
+ return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
+ .setRegionId(regionId)
+ .setStatus(status)
+ .build();
+ } else if (parts.length == 5) {
+ long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
+ assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
+
+ long logSegmentId = Long.parseLong(parts[1]);
+ long firstTxId = Long.parseLong(parts[2]);
+ long lastTxId = Long.parseLong(parts[3]);
+ long completionTime = Long.parseLong(parts[4]);
+ return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
+ .setInprogress(false)
+ .setLastTxId(lastTxId)
+ .setCompletionTime(completionTime)
+ .setRecordCount((int) recordCount)
+ .setRegionId(regionId)
+ .setStatus(status)
+ .build();
+ } else {
+ throw new IOException("Invalid log segment metadata : "
+ + new String(data, UTF_8));
+ }
+ }
+
+ static LogSegmentMetadata parseDataV2(String path, byte[] data, String[] parts)
+ throws IOException {
+ long versionStatusCount = Long.parseLong(parts[0]);
+
+ long version = versionStatusCount & METADATA_VERSION_MASK;
+ assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
+ assert (2 == version);
+
+ LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO;
+
+ int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
+ assert (regionId >= 0 && regionId <= 0xf);
+
+ long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
+ assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
+
+ if (parts.length == 4) {
+ long logSegmentId = Long.parseLong(parts[1]);
+ long txId = Long.parseLong(parts[2]);
+ long logSegmentSequenceNumber = Long.parseLong(parts[3]);
+ return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
+ .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+ .setRegionId(regionId)
+ .setStatus(status)
+ .build();
+ } else if (parts.length == 8) {
+ long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
+ assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
+
+ long logSegmentId = Long.parseLong(parts[1]);
+ long firstTxId = Long.parseLong(parts[2]);
+ long lastTxId = Long.parseLong(parts[3]);
+ long completionTime = Long.parseLong(parts[4]);
+ long logSegmentSequenceNumber = Long.parseLong(parts[5]);
+ long lastEntryId = Long.parseLong(parts[6]);
+ long lastSlotId = Long.parseLong(parts[7]);
+ return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
+ .setInprogress(false)
+ .setLastTxId(lastTxId)
+ .setCompletionTime(completionTime)
+ .setRecordCount((int) recordCount)
+ .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+ .setLastEntryId(lastEntryId)
+ .setLastSlotId(lastSlotId)
+ .setRegionId(regionId)
+ .setStatus(status)
+ .build();
+ } else {
+ throw new IOException("Invalid logsegment metadata : "
+ + new String(data, UTF_8));
+ }
+
+ }
+
+ static LogSegmentMetadata parseDataVersionsWithMinActiveDLSN(String path, byte[] data, String[] parts)
+ throws IOException {
+ long versionStatusCount = Long.parseLong(parts[0]);
+
+ long version = versionStatusCount & METADATA_VERSION_MASK;
+ assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
+ assert (LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version &&
+ LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version);
+
+ LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
+
+ int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
+ assert (regionId >= 0 && regionId <= 0xf);
+
+ long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
+ assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
+
+ if (parts.length == 6) {
+ long logSegmentId = Long.parseLong(parts[1]);
+ long txId = Long.parseLong(parts[2]);
+ long logSegmentSequenceNumber = Long.parseLong(parts[3]);
+ long minActiveEntryId = Long.parseLong(parts[4]);
+ long minActiveSlotId = Long.parseLong(parts[5]);
+
+ LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
+ .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+ .setMinActiveEntryId(minActiveEntryId)
+ .setMinActiveSlotId(minActiveSlotId)
+ .setRegionId(regionId)
+ .setStatus(status);
+ if (supportsEnvelopedEntries((int) version)) {
+ builder = builder.setEnvelopeEntries(true);
+ }
+ return builder.build();
+ } else if (parts.length == 10) {
+ long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
+ assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
+
+ long logSegmentId = Long.parseLong(parts[1]);
+ long firstTxId = Long.parseLong(parts[2]);
+ long lastTxId = Long.parseLong(parts[3]);
+ long completionTime = Long.parseLong(parts[4]);
+ long logSegmentSequenceNumber = Long.parseLong(parts[5]);
+ long lastEntryId = Long.parseLong(parts[6]);
+ long lastSlotId = Long.parseLong(parts[7]);
+ long minActiveEntryId = Long.parseLong(parts[8]);
+ long minActiveSlotId = Long.parseLong(parts[9]);
+ LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
+ .setInprogress(false)
+ .setLastTxId(lastTxId)
+ .setCompletionTime(completionTime)
+ .setRecordCount((int) recordCount)
+ .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+ .setLastEntryId(lastEntryId)
+ .setLastSlotId(lastSlotId)
+ .setMinActiveEntryId(minActiveEntryId)
+ .setMinActiveSlotId(minActiveSlotId)
+ .setRegionId(regionId)
+ .setStatus(status);
+ if (supportsEnvelopedEntries((int) version)) {
+ builder = builder.setEnvelopeEntries(true);
+ }
+ return builder.build();
+ } else {
+ throw new IOException("Invalid logsegment metadata : "
+ + new String(data, UTF_8));
+ }
+
+ }
+
+ static LogSegmentMetadata parseDataVersionsWithSequenceId(String path, byte[] data, String[] parts)
+ throws IOException {
+ long versionStatusCount = Long.parseLong(parts[0]);
+
+ long version = versionStatusCount & METADATA_VERSION_MASK;
+ assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
+ assert (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version &&
+ LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version);
+
+ LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
+
+ int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
+ assert (regionId >= 0 && regionId <= 0xf);
+
+ long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
+ assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
+
+ if (parts.length == 7) {
+ long logSegmentId = Long.parseLong(parts[1]);
+ long txId = Long.parseLong(parts[2]);
+ long logSegmentSequenceNumber = Long.parseLong(parts[3]);
+ long minActiveEntryId = Long.parseLong(parts[4]);
+ long minActiveSlotId = Long.parseLong(parts[5]);
+ long startSequenceId = Long.parseLong(parts[6]);
+
+ LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
+ .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+ .setMinActiveEntryId(minActiveEntryId)
+ .setMinActiveSlotId(minActiveSlotId)
+ .setRegionId(regionId)
+ .setStatus(status)
+ .setStartSequenceId(startSequenceId)
+ .setEnvelopeEntries(true);
+ return builder.build();
+ } else if (parts.length == 11) {
+ long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
+ assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
+
+ long logSegmentId = Long.parseLong(parts[1]);
+ long firstTxId = Long.parseLong(parts[2]);
+ long lastTxId = Long.parseLong(parts[3]);
+ long completionTime = Long.parseLong(parts[4]);
+ long logSegmentSequenceNumber = Long.parseLong(parts[5]);
+ long lastEntryId = Long.parseLong(parts[6]);
+ long lastSlotId = Long.parseLong(parts[7]);
+ long minActiveEntryId = Long.parseLong(parts[8]);
+ long minActiveSlotId = Long.parseLong(parts[9]);
+ long startSequenceId = Long.parseLong(parts[10]);
+ LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
+ .setInprogress(false)
+ .setLastTxId(lastTxId)
+ .setCompletionTime(completionTime)
+ .setRecordCount((int) recordCount)
+ .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+ .setLastEntryId(lastEntryId)
+ .setLastSlotId(lastSlotId)
+ .setMinActiveEntryId(minActiveEntryId)
+ .setMinActiveSlotId(minActiveSlotId)
+ .setRegionId(regionId)
+ .setStatus(status)
+ .setStartSequenceId(startSequenceId)
+ .setEnvelopeEntries(true);
+ return builder.build();
+ } else {
+ throw new IOException("Invalid log segment metadata : "
+ + new String(data, UTF_8));
+ }
+ }
+
+ public static LogSegmentMetadata parseData(String path, byte[] data)
+ throws IOException {
+ return parseData(path, data, false);
+ }
+
+ static LogSegmentMetadata parseData(String path, byte[] data, boolean skipMinVersionCheck) throws IOException {
+ String[] parts = new String(data, UTF_8).split(";");
+ long version;
+ try {
+ version = Long.parseLong(parts[0]) & METADATA_VERSION_MASK;
+ } catch (Exception exc) {
+ throw new IOException("Invalid ledger entry, "
+ + new String(data, UTF_8));
+ }
+
+ if (!skipMinVersionCheck && version < LogSegmentMetadata.LEDGER_METADATA_OLDEST_SUPPORTED_VERSION) {
+ throw new UnsupportedMetadataVersionException("Ledger metadata version '" + version + "' is no longer supported: "
+ + new String(data, UTF_8));
+ }
+
+ if (version > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION) {
+ throw new UnsupportedMetadataVersionException("Metadata version '" + version + "' is higher than the highest supported version : "
+ + new String(data, UTF_8));
+ }
+
+ if (LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value == version) {
+ return parseDataV1(path, data, parts);
+ } else if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value == version) {
+ return parseDataV2(path, data, parts);
+ } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version &&
+ LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version) {
+ return parseDataVersionsWithMinActiveDLSN(path, data, parts);
+ } else {
+ assert(version >= LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
+ return parseDataVersionsWithSequenceId(path, data, parts);
+ }
+ }
+
+ public String getFinalisedData() {
+ return getFinalisedData(this.version);
+ }
+
+ public String getFinalisedData(LogSegmentMetadataVersion version) {
+ String finalisedData;
+ final long logSegmentSeqNo = getLogSegmentSequenceNumber();
+ final long lastEntryId = getLastEntryId();
+ final long lastSlotId = getLastSlotId();
+ final long minActiveEntryId = minActiveDLSN.getEntryId();
+ final long minActiveSlotId = minActiveDLSN.getSlotId();
+
+ if (LogSegmentMetadataVersion.VERSION_V1_ORIGINAL == version) {
+ if (inprogress) {
+ finalisedData = String.format("%d;%d;%d",
+ version.value, logSegmentId, firstTxId);
+ } else {
+ long versionAndCount = ((long) version.value) | ((long)recordCount << LOGRECORD_COUNT_SHIFT);
+ finalisedData = String.format("%d;%d;%d;%d;%d",
+ versionAndCount, logSegmentId, firstTxId, lastTxId, completionTime);
+ }
+ } else {
+ long versionStatusCount = ((long) version.value);
+ versionStatusCount |= ((status & METADATA_STATUS_BIT_MAX) << STATUS_BITS_SHIFT);
+ versionStatusCount |= (((long) regionId & MAX_REGION_ID) << REGION_SHIFT);
+ if (!inprogress) {
+ versionStatusCount |= ((long)recordCount << LOGRECORD_COUNT_SHIFT);
+ }
+ if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO == version) {
+ if (inprogress) {
+ finalisedData = String.format("%d;%d;%d;%d",
+ versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo);
+ } else {
+ finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d",
+ versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
+ logSegmentSeqNo, lastEntryId, lastSlotId);
+ }
+ } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version.value &&
+ LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version.value) {
+ if (inprogress) {
+ finalisedData = String.format("%d;%d;%d;%d;%d;%d",
+ versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId);
+ } else {
+ finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
+ versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
+ logSegmentSeqNo, lastEntryId, lastSlotId, minActiveEntryId, minActiveSlotId);
+ }
+ } else if (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version.value &&
+ LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version.value) {
+ if (inprogress) {
+ finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d",
+ versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId, startSequenceId);
+ } else {
+ finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
+ versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
+ logSegmentSeqNo, lastEntryId, lastSlotId, minActiveEntryId, minActiveSlotId, startSequenceId);
+ }
+ } else {
+ throw new IllegalStateException("Unsupported log segment ledger metadata version '" + version + "'");
+ }
+ }
+ return finalisedData;
+ }
+
+ String getSegmentName() {
+ String[] parts = this.zkPath.split("/");
+ if (parts.length <= 0) {
+ throw new IllegalStateException("ZK Path is not valid");
+ }
+ return parts[parts.length - 1];
+ }
+
+ public void write(ZooKeeperClient zkc)
+ throws IOException, KeeperException.NodeExistsException {
+ String finalisedData = getFinalisedData(version);
+ try {
+ zkc.get().create(zkPath, finalisedData.getBytes(UTF_8),
+ zkc.getDefaultACL(), CreateMode.PERSISTENT);
+ } catch (KeeperException.NodeExistsException nee) {
+ throw nee;
+ } catch (InterruptedException ie) {
+ throw new DLInterruptedException("Interrupted on creating ledger znode " + zkPath, ie);
+ } catch (Exception e) {
+ LOG.error("Error creating ledger znode {}", zkPath, e);
+ throw new IOException("Error creating ledger znode " + zkPath);
+ }
+ }
+
+ boolean checkEquivalence(ZooKeeperClient zkc, String path) {
+ try {
+ LogSegmentMetadata other = FutureUtils.result(read(zkc, path));
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Verifying {} against {}", this, other);
+ }
+
+ boolean retVal;
+
+ // All fields may not be comparable so only compare the ones
+ // that can be compared
+ // completionTime is set when a node is finalized, so that
+ // cannot be compared
+ // if the node is inprogress, don't compare the lastTxId either
+ if (this.getLogSegmentSequenceNumber() != other.getLogSegmentSequenceNumber() ||
+ this.logSegmentId != other.logSegmentId ||
+ this.firstTxId != other.firstTxId) {
+ retVal = false;
+ } else if (this.inprogress) {
+ retVal = other.inprogress;
+ } else {
+ retVal = (!other.inprogress && (this.lastTxId == other.lastTxId));
+ }
+
+ if (!retVal) {
+ LOG.warn("Equivalence check failed between {} and {}", this, other);
+ }
+
+ return retVal;
+ } catch (Exception e) {
+ LOG.error("Could not check equivalence between:" + this + " and data in " + path, e);
+ return false;
+ }
+ }
+
+ public boolean equals(Object o) {
+ if (!(o instanceof LogSegmentMetadata)) {
+ return false;
+ }
+ LogSegmentMetadata ol = (LogSegmentMetadata) o;
+ return getLogSegmentSequenceNumber() == ol.getLogSegmentSequenceNumber()
+ && logSegmentId == ol.logSegmentId
+ && firstTxId == ol.firstTxId
+ && lastTxId == ol.lastTxId
+ && version == ol.version
+ && completionTime == ol.completionTime
+ && Objects.equal(lastDLSN, ol.lastDLSN)
+ && Objects.equal(minActiveDLSN, ol.minActiveDLSN)
+ && startSequenceId == ol.startSequenceId
+ && status == ol.status;
+ }
+
+ public int hashCode() {
+ int hash = 1;
+ hash = hash * 31 + (int) logSegmentId;
+ hash = hash * 31 + (int) firstTxId;
+ hash = hash * 31 + (int) lastTxId;
+ hash = hash * 31 + version.value;
+ hash = hash * 31 + (int) completionTime;
+ hash = hash * 31 + (int) getLogSegmentSequenceNumber();
+ return hash;
+ }
+
+ public String toString() {
+ return "[LogSegmentId:" + logSegmentId +
+ ", firstTxId:" + firstTxId +
+ ", lastTxId:" + lastTxId +
+ ", version:" + version +
+ ", completionTime:" + completionTime +
+ ", recordCount:" + recordCount +
+ ", regionId:" + regionId +
+ ", status:" + status +
+ ", logSegmentSequenceNumber:" + getLogSegmentSequenceNumber() +
+ ", lastEntryId:" + getLastEntryId() +
+ ", lastSlotId:" + getLastSlotId() +
+ ", inprogress:" + inprogress +
+ ", minActiveDLSN:" + minActiveDLSN +
+ ", startSequenceId:" + startSequenceId +
+ "]";
+ }
+
+ public Mutator mutator() {
+ return new Mutator(this);
+ }
+
+
+ //
+ // Version Checking Utilities
+ //
+
+ public boolean supportsLogSegmentSequenceNo() {
+ return supportsLogSegmentSequenceNo(version.value);
+ }
+
+ /**
+ * Whether the provided version supports log segment sequence number.
+ *
+ * @param version
+ * log segment metadata version
+ * @return true if this log segment supports log segment sequence number.
+ */
+ public static boolean supportsLogSegmentSequenceNo(int version) {
+ return version >= LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value;
+ }
+
+ /**
+ * Whether the provided version supports enveloping entries before writing to bookkeeper.
+ *
+ * @param version
+ * log segment metadata version
+ * @return true if this log segment supports enveloping entries
+ */
+ public static boolean supportsEnvelopedEntries(int version) {
+ return version >= LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value;
+ }
+
+ public boolean supportsSequenceId() {
+ return supportsSequenceId(version.value);
+ }
+
+ /**
+ * Whether the provided version supports sequence id.
+ *
+ * @param version
+ * log segment metadata version
+ * @return true if the log segment support sequence id.
+ */
+ public static boolean supportsSequenceId(int version) {
+ return version >= LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java
new file mode 100644
index 0000000..8a4a30b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java
@@ -0,0 +1,78 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.io.Abortable;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+/*
+* A generic interface class to support writing log records into
+* a persistent distributed log.
+*/
+public interface LogWriter extends Closeable, Abortable {
+ /**
+ * Write a log record to the stream.
+ *
+ * @param record single log record
+ * @throws IOException
+ */
+ public void write(LogRecord record) throws IOException;
+
+
+ /**
+ * Write a list of log records to the stream.
+ *
+ * @param records list of log records
+ * @throws IOException
+ */
+ @Deprecated
+ public int writeBulk(List<LogRecord> records) throws IOException;
+
+ /**
+ * All data that has been written to the stream so far will be sent to
+ * persistent storage.
+ * The transmission is asynchronous and new data can be still written to the
+ * stream while flushing is performed.
+ *
+ * TODO: rename this to flush()
+ */
+ public long setReadyToFlush() throws IOException;
+
+ /**
+ * Flush and sync all data that is ready to be flush
+ * {@link #setReadyToFlush()} into underlying persistent store.
+ * @throws IOException
+ *
+ * TODO: rename this to commit()
+ */
+ public long flushAndSync() throws IOException;
+
+ /**
+ * Flushes all the data up to this point,
+ * adds the end of stream marker and marks the stream
+ * as read-only in the metadata. No appends to the
+ * stream will be allowed after this point
+ *
+ * @throws IOException
+ */
+ public void markEndOfStream() throws IOException;
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java
new file mode 100644
index 0000000..a76f547
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java
@@ -0,0 +1,72 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.util.DLUtils;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+
+/**
+ * Utility class for storing and reading max ledger sequence number
+ */
+class MaxLogSegmentSequenceNo {
+
+ Version version;
+ long maxSeqNo;
+
+ MaxLogSegmentSequenceNo(Versioned<byte[]> logSegmentsData) {
+ if (null != logSegmentsData
+ && null != logSegmentsData.getValue()
+ && null != logSegmentsData.getVersion()) {
+ version = logSegmentsData.getVersion();
+ try {
+ maxSeqNo = DLUtils.deserializeLogSegmentSequenceNumber(logSegmentsData.getValue());
+ } catch (NumberFormatException nfe) {
+ maxSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
+ }
+ } else {
+ maxSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
+ if (null != logSegmentsData && null != logSegmentsData.getVersion()) {
+ version = logSegmentsData.getVersion();
+ } else {
+ throw new IllegalStateException("Invalid MaxLogSegmentSequenceNo found - " + logSegmentsData);
+ }
+ }
+ }
+
+ synchronized Version getVersion() {
+ return version;
+ }
+
+ synchronized long getSequenceNumber() {
+ return maxSeqNo;
+ }
+
+ synchronized MaxLogSegmentSequenceNo update(Version version, long logSegmentSeqNo) {
+ if (version.compare(this.version) == Version.Occurred.AFTER) {
+ this.version = version;
+ this.maxSeqNo = logSegmentSeqNo;
+ }
+ return this;
+ }
+
+ public synchronized Versioned<Long> getVersionedData(long seqNo) {
+ return new Versioned<Long>(seqNo, version);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java
new file mode 100644
index 0000000..8f077e2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java
@@ -0,0 +1,72 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.util.DLUtils;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class for storing and reading
+ * the max seen txid in zookeeper
+ */
+class MaxTxId {
+ static final Logger LOG = LoggerFactory.getLogger(MaxTxId.class);
+
+ private Version version;
+ private long currentMax;
+
+ MaxTxId(Versioned<byte[]> maxTxIdData) {
+ if (null != maxTxIdData
+ && null != maxTxIdData.getValue()
+ && null != maxTxIdData.getVersion()) {
+ this.version = maxTxIdData.getVersion();
+ try {
+ this.currentMax = DLUtils.deserializeTransactionId(maxTxIdData.getValue());
+ } catch (NumberFormatException e) {
+ LOG.warn("Invalid txn id stored in {}", e);
+ this.currentMax = DistributedLogConstants.INVALID_TXID;
+ }
+ } else {
+ this.currentMax = DistributedLogConstants.INVALID_TXID;
+ if (null != maxTxIdData && null != maxTxIdData.getVersion()) {
+ this.version = maxTxIdData.getVersion();
+ } else {
+ throw new IllegalStateException("Invalid MaxTxId found - " + maxTxIdData);
+ }
+ }
+ }
+
+ synchronized void update(Version version, long txId) {
+ if (version.compare(this.version) == Version.Occurred.AFTER) {
+ this.version = version;
+ this.currentMax = txId;
+ }
+ }
+
+ synchronized long get() {
+ return currentMax;
+ }
+
+ public synchronized Versioned<Long> getVersionedData(long txId) {
+ return new Versioned<Long>(Math.max(txId, get()), version);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java
----------------------------------------------------------------------
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java
new file mode 100644
index 0000000..3d1d601
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java
@@ -0,0 +1,43 @@
+/**
+ * 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 org.apache.distributedlog;
+
+import org.apache.distributedlog.io.AsyncCloseable;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public interface MetadataAccessor extends Closeable, AsyncCloseable {
+ /**
+ * Get the name of the stream managed by this log manager
+ * @return streamName
+ */
+ public String getStreamName();
+
+ public void createOrUpdateMetadata(byte[] metadata) throws IOException;
+
+ public void deleteMetadata() throws IOException;
+
+ public byte[] getMetadata() throws IOException;
+
+ /**
+ * Close the distributed log metadata, freeing any resources it may hold.
+ */
+ public void close() throws IOException;
+
+}